You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by ti...@apache.org on 2016/05/26 19:19:17 UTC

[7/8] incubator-asterixdb git commit: ASTERIXDB-1220: print nested values using visitors

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinter.java
deleted file mode 100644
index 61b73fc..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinter.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.PrintStream;
-
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ANullPrinter implements IPrinter {
-
-    public static final ANullPrinter INSTANCE = new ANullPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("null");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinterFactory.java
index e37c06a..d9cb1ff 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullPrinterFactory.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +28,10 @@ public class ANullPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ANullPrinterFactory INSTANCE = new ANullPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> ps.print("null");
+
     @Override
     public IPrinter createPrinter() {
-        return ANullPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullableFieldPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullableFieldPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullableFieldPrinterFactory.java
index 010e968..784bd0b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullableFieldPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ANullableFieldPrinterFactory.java
@@ -61,8 +61,6 @@ public class ANullableFieldPrinterFactory implements IPrinterFactory {
                     fieldPrinter.print(b, s, l, ps);
                 }
             }
-
         };
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinter.java
deleted file mode 100644
index 27aa754..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinter.java
+++ /dev/null
@@ -1,176 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.EnumDeserializer;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class AObjectPrinter implements IPrinter {
-
-    public static final AObjectPrinter INSTANCE = new AObjectPrinter();
-
-    private IPrinter recordPrinter = new ARecordPrinterFactory(null).createPrinter();
-    private IPrinter orderedlistPrinter = new AOrderedlistPrinterFactory(null).createPrinter();
-    private IPrinter unorderedListPrinter = new AUnorderedlistPrinterFactory(null).createPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b[s]);
-        switch (typeTag) {
-            case INT8: {
-                AInt8Printer.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case INT16: {
-                AInt16Printer.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case INT32: {
-                AInt32Printer.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case INT64: {
-                AInt64Printer.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case MISSING:
-            case NULL: {
-                ANullPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case BOOLEAN: {
-                ABooleanPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case FLOAT: {
-                AFloatPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case DOUBLE: {
-                ADoublePrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case DATE: {
-                ADatePrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case TIME: {
-                ATimePrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case DATETIME: {
-                ADateTimePrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case DURATION: {
-                ADurationPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case YEARMONTHDURATION: {
-                AYearMonthDurationPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case DAYTIMEDURATION: {
-                ADayTimeDurationPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case INTERVAL: {
-                AIntervalPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case POINT: {
-                APointPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case POINT3D: {
-                APoint3DPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case LINE: {
-                ALinePrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case POLYGON: {
-                APolygonPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case RECTANGLE: {
-                ARectanglePrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case CIRCLE: {
-                ACirclePrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case STRING: {
-                AStringPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case BINARY: {
-                ABinaryHexPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case RECORD: {
-                this.recordPrinter.init();
-                recordPrinter.print(b, s, l, ps);
-                break;
-            }
-            case ORDEREDLIST: {
-                this.orderedlistPrinter.init();
-                orderedlistPrinter.print(b, s, l, ps);
-                break;
-            }
-            case UNORDEREDLIST: {
-                this.unorderedListPrinter.init();
-                unorderedListPrinter.print(b, s, l, ps);
-                break;
-            }
-            case UUID: {
-                AUUIDPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case SHORTWITHOUTTYPEINFO: {
-                ShortWithoutTypeInfoPrinter.INSTANCE.print(b, s, l, ps);
-                break;
-            }
-            case ANY:
-            case BITARRAY:
-            case ENUM:
-            case SPARSERECORD:
-            case SYSTEM_NULL:
-            case TYPE:
-            case UINT16:
-            case UINT32:
-            case UINT64:
-            case UINT8:
-            case UNION:
-                // These are internal types and do not need a printer.
-                throw new NotImplementedException("No printer for type " + typeTag);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
index 2a86f88..db3480d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AObjectPrinterFactory.java
@@ -18,17 +18,148 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.common.exceptions.AsterixException;
+import org.apache.asterix.om.pointables.AListVisitablePointable;
+import org.apache.asterix.om.pointables.ARecordVisitablePointable;
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.pointables.printer.IPrintVisitor;
+import org.apache.asterix.om.pointables.printer.adm.APrintVisitor;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.EnumDeserializer;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class AObjectPrinterFactory implements IPrinterFactory {
 
     private static final long serialVersionUID = 1L;
     public static final AObjectPrinterFactory INSTANCE = new AObjectPrinterFactory();
 
+    public static boolean printFlatValue(ATypeTag typeTag, byte[] b, int s, int l, PrintStream ps)
+            throws HyracksDataException {
+        switch (typeTag) {
+            case INT8:
+                AInt8PrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case INT16:
+                AInt16PrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case INT32:
+                AInt32PrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case INT64:
+                AInt64PrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case MISSING:
+            case NULL:
+                ANullPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case BOOLEAN:
+                ABooleanPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case FLOAT:
+                AFloatPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case DOUBLE:
+                ADoublePrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case DATE:
+                ADatePrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case TIME:
+                ATimePrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case DATETIME:
+                ADateTimePrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case DURATION:
+                ADurationPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case YEARMONTHDURATION:
+                AYearMonthDurationPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case DAYTIMEDURATION:
+                ADayTimeDurationPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case INTERVAL:
+                AIntervalPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case POINT:
+                APointPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case POINT3D:
+                APoint3DPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case LINE:
+                ALinePrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case POLYGON:
+                APolygonPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case RECTANGLE:
+                ARectanglePrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case CIRCLE:
+                ACirclePrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case STRING:
+                AStringPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case BINARY:
+                ABinaryHexPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case UUID:
+                AUUIDPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case SHORTWITHOUTTYPEINFO:
+                ShortWithoutTypeInfoPrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            default:
+                return false;
+        }
+    }
+
     @Override
     public IPrinter createPrinter() {
-        return AObjectPrinter.INSTANCE;
-    }
+        final ARecordVisitablePointable rPointable = new ARecordVisitablePointable(
+                DefaultOpenFieldType.NESTED_OPEN_RECORD_TYPE);
+        final AListVisitablePointable olPointable = new AListVisitablePointable(
+                DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE);
+        final AListVisitablePointable ulPointable = new AListVisitablePointable(
+                DefaultOpenFieldType.NESTED_OPEN_AUNORDERED_LIST_TYPE);
+        final Pair<PrintStream, ATypeTag> streamTag = new Pair<>(null, null);
+
+        final IPrintVisitor visitor = new APrintVisitor();
 
+        return (byte[] b, int s, int l, PrintStream ps) -> {
+            ATypeTag typeTag = EnumDeserializer.ATYPETAGDESERIALIZER.deserialize(b[s]);
+            if (!printFlatValue(typeTag, b, s, l, ps)) {
+                streamTag.first = ps;
+                streamTag.second = typeTag;
+                try {
+                    switch (typeTag) {
+                        case RECORD:
+                            rPointable.set(b, s, l);
+                            visitor.visit(rPointable, streamTag);
+                            break;
+                        case ORDEREDLIST:
+                            olPointable.set(b, s, l);
+                            visitor.visit(olPointable, streamTag);
+                            break;
+                        case UNORDEREDLIST:
+                            ulPointable.set(b, s, l);
+                            visitor.visit(ulPointable, streamTag);
+                            break;
+                        default:
+                            throw new HyracksDataException("No printer for type " + typeTag);
+                    }
+                } catch (AsterixException e) {
+                    throw new HyracksDataException(e);
+                }
+            }
+        };
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AOrderedlistPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AOrderedlistPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AOrderedlistPrinterFactory.java
index 7e8864f..36cab4e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AOrderedlistPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AOrderedlistPrinterFactory.java
@@ -36,7 +36,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class AOrderedlistPrinterFactory implements IPrinterFactory {
 
     private static final long serialVersionUID = 1L;
-    private AOrderedListType orderedlistType;
+    private final AOrderedListType orderedlistType;
 
     public AOrderedlistPrinterFactory(AOrderedListType orderedlistType) {
         this.orderedlistType = orderedlistType;
@@ -44,16 +44,14 @@ public class AOrderedlistPrinterFactory implements IPrinterFactory {
 
     @Override
     public IPrinter createPrinter() {
-
-        PointableAllocator allocator = new PointableAllocator();
-        final IAType inputType = orderedlistType == null ? DefaultOpenFieldType
-                .getDefaultOpenFieldType(ATypeTag.ORDEREDLIST) : orderedlistType;
+        final PointableAllocator allocator = new PointableAllocator();
+        final IAType inputType = orderedlistType == null
+                ? DefaultOpenFieldType.getDefaultOpenFieldType(ATypeTag.ORDEREDLIST) : orderedlistType;
         final IVisitablePointable listAccessor = allocator.allocateListValue(inputType);
         final APrintVisitor printVisitor = new APrintVisitor();
-        final Pair<PrintStream, ATypeTag> arg = new Pair<PrintStream, ATypeTag>(null, null);
+        final Pair<PrintStream, ATypeTag> arg = new Pair<>(null, null);
 
         return new IPrinter() {
-
             @Override
             public void init() {
                 arg.second = inputType.getTypeTag();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinter.java
deleted file mode 100644
index 5912cc4..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class APoint3DPrinter implements IPrinter {
-
-    public static final APoint3DPrinter INSTANCE = new APoint3DPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("point3d(\"");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
-        ps.print(",");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 9));
-        ps.print(",");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 17));
-        ps.print("\")");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinterFactory.java
index 692e17c..cca84eb 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APoint3DPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +29,18 @@ public class APoint3DPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final APoint3DPrinterFactory INSTANCE = new APoint3DPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("point3d(\"");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
+        ps.print(",");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 9));
+        ps.print(",");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 17));
+        ps.print("\")");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return APoint3DPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinter.java
deleted file mode 100644
index 56caf52..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinter.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class APointPrinter implements IPrinter {
-
-    public static final APointPrinter INSTANCE = new APointPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("point(\"");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
-        ps.print(",");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 9));
-        ps.print("\")");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinterFactory.java
index eba9ea8..ca0f8e9 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APointPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +29,16 @@ public class APointPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final APointPrinterFactory INSTANCE = new APointPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("point(\"");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
+        ps.print(",");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 9));
+        ps.print("\")");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return APointPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinter.java
deleted file mode 100644
index 2f592f8..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinter.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class APolygonPrinter implements IPrinter {
-
-    public static final APolygonPrinter INSTANCE = new APolygonPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        short numberOfPoints = AInt16SerializerDeserializer.getShort(b, s + 1);
-        s += 3;
-        ps.print("polygon(\"");
-        for (int i = 0; i < numberOfPoints; i++) {
-            if (i > 0)
-                ps.print(" ");
-            ps.print(ADoubleSerializerDeserializer.getDouble(b, s));
-            ps.print(",");
-            ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 8));
-            s += 16;
-        }
-        ps.print("\")");
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinterFactory.java
index 3f4c7f1..9280fee 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/APolygonPrinterFactory.java
@@ -18,6 +18,10 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt16SerializerDeserializer;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +30,24 @@ public class APolygonPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final APolygonPrinterFactory INSTANCE = new APolygonPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        short numberOfPoints = AInt16SerializerDeserializer.getShort(b, s + 1);
+        s += 3;
+        ps.print("polygon(\"");
+        for (int i = 0; i < numberOfPoints; i++) {
+            if (i > 0) {
+                ps.print(" ");
+            }
+            ps.print(ADoubleSerializerDeserializer.getDouble(b, s));
+            ps.print(",");
+            ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 8));
+            s += 16;
+        }
+        ps.print("\")");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return APolygonPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARecordPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARecordPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARecordPrinterFactory.java
index 7340db3..6d5180e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARecordPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARecordPrinterFactory.java
@@ -44,16 +44,14 @@ public class ARecordPrinterFactory implements IPrinterFactory {
 
     @Override
     public IPrinter createPrinter() {
-
-        PointableAllocator allocator = new PointableAllocator();
+        final PointableAllocator allocator = new PointableAllocator();
         final IAType inputType = recType == null ? DefaultOpenFieldType.getDefaultOpenFieldType(ATypeTag.RECORD)
                 : recType;
         final IVisitablePointable recAccessor = allocator.allocateRecordValue(inputType);
         final APrintVisitor printVisitor = new APrintVisitor();
-        final Pair<PrintStream, ATypeTag> arg = new Pair<PrintStream, ATypeTag>(null, null);
+        final Pair<PrintStream, ATypeTag> arg = new Pair<>(null, null);
 
         return new IPrinter() {
-
             @Override
             public void init() {
                 arg.second = inputType.getTypeTag();

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinter.java
deleted file mode 100644
index 54d3bf3..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinter.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ARectanglePrinter implements IPrinter {
-
-    public static final ARectanglePrinter INSTANCE = new ARectanglePrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("rectangle(\"");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
-        ps.print(",");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 9));
-        ps.print(" ");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 17));
-        ps.print(",");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 25));
-        ps.print("\")");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinterFactory.java
index 1ba973a..4d10048 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ARectanglePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +29,20 @@ public class ARectanglePrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ARectanglePrinterFactory INSTANCE = new ARectanglePrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("rectangle(\"");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
+        ps.print(",");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 9));
+        ps.print(" ");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 17));
+        ps.print(",");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 25));
+        ps.print("\")");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return ARectanglePrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinter.java
deleted file mode 100644
index ac1f660..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.IOException;
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class AStringPrinter implements IPrinter {
-
-    public static final AStringPrinter INSTANCE = new AStringPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        try {
-            // ADM uses same escape semantics as JSON for strings
-            PrintTools.writeUTF8StringAsJSON(b, s + 1, l - 1, ps);
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinterFactory.java
index 729f962..c24b5bc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AStringPrinterFactory.java
@@ -18,17 +18,30 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public class AStringPrinterFactory implements IPrinterFactory {
 
     private static final long serialVersionUID = 1L;
     public static final AStringPrinterFactory INSTANCE = new AStringPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        try {
+            // ADM uses same escape semantics as JSON for strings
+            PrintTools.writeUTF8StringAsJSON(b, s + 1, l - 1, ps);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return AStringPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinter.java
deleted file mode 100644
index 987a666..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinter.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ATimePrinter implements IPrinter {
-
-    public static final ATimePrinter INSTANCE = new ATimePrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("time(\"");
-        PrintTools.printTimeString(b, s, l, ps);
-        ps.print("\")");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinterFactory.java
index 69abd41..f9ad18e 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ATimePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +29,14 @@ public class ATimePrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ATimePrinterFactory INSTANCE = new ATimePrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("time(\"");
+        PrintTools.printTimeString(b, s, l, ps);
+        ps.print("\")");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return ATimePrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinter.java
deleted file mode 100644
index 786984f..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinter.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.om.base.AUUID;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class AUUIDPrinter implements IPrinter {
-
-    public static final AUUIDPrinter INSTANCE = new AUUIDPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        StringBuilder buf = new StringBuilder(AUUID.UUID_CHARS + 8);
-        buf.append("uuid(\"");
-        AUUID.appendLiteralOnly(b, s + 1, buf).append("\")");
-        ps.print(buf.toString());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java
index eeed0b8..41da3bd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUUIDPrinterFactory.java
@@ -19,6 +19,9 @@
 
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.om.base.AUUID;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -28,9 +31,15 @@ public class AUUIDPrinterFactory implements IPrinterFactory {
 
     public static final AUUIDPrinterFactory INSTANCE = new AUUIDPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        StringBuilder buf = new StringBuilder(AUUID.UUID_CHARS + 8);
+        buf.append("uuid(\"");
+        AUUID.appendLiteralOnly(b, s + 1, buf).append("\")");
+        ps.print(buf.toString());
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return AUUIDPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnionPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnionPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnionPrinterFactory.java
index 12baa16..17f82d1 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnionPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnionPrinterFactory.java
@@ -32,7 +32,6 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class AUnionPrinterFactory implements IPrinterFactory {
 
     private static final long serialVersionUID = 1L;
-
     private AUnionType unionType;
 
     public AUnionPrinterFactory(AUnionType unionType) {
@@ -41,9 +40,7 @@ public class AUnionPrinterFactory implements IPrinterFactory {
 
     @Override
     public IPrinter createPrinter() {
-
         return new IPrinter() {
-
             private IPrinter[] printers;
             private List<IAType> unionList;
 
@@ -73,5 +70,4 @@ public class AUnionPrinterFactory implements IPrinterFactory {
             }
         };
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnorderedlistPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnorderedlistPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnorderedlistPrinterFactory.java
index c185881..7d4c18a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnorderedlistPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AUnorderedlistPrinterFactory.java
@@ -20,6 +20,7 @@ package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
 import java.io.PrintStream;
 
+import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.om.pointables.PointableAllocator;
 import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
 import org.apache.asterix.om.pointables.base.IVisitablePointable;
@@ -35,7 +36,7 @@ import org.apache.hyracks.api.exceptions.HyracksDataException;
 public class AUnorderedlistPrinterFactory implements IPrinterFactory {
 
     private static final long serialVersionUID = 1L;
-    private AUnorderedListType unorderedlistType;
+    private final AUnorderedListType unorderedlistType;
 
     public AUnorderedlistPrinterFactory(AUnorderedListType unorderedlistType) {
         this.unorderedlistType = unorderedlistType;
@@ -43,16 +44,14 @@ public class AUnorderedlistPrinterFactory implements IPrinterFactory {
 
     @Override
     public IPrinter createPrinter() {
-
-        PointableAllocator allocator = new PointableAllocator();
+        final PointableAllocator allocator = new PointableAllocator();
         final IAType inputType = unorderedlistType == null
                 ? DefaultOpenFieldType.getDefaultOpenFieldType(ATypeTag.UNORDEREDLIST) : unorderedlistType;
         final IVisitablePointable listAccessor = allocator.allocateListValue(inputType);
         final APrintVisitor printVisitor = new APrintVisitor();
-        final Pair<PrintStream, ATypeTag> arg = new Pair<PrintStream, ATypeTag>(null, null);
+        final Pair<PrintStream, ATypeTag> arg = new Pair<>(null, null);
 
         return new IPrinter() {
-
             @Override
             public void init() {
                 arg.second = inputType.getTypeTag();
@@ -64,7 +63,7 @@ public class AUnorderedlistPrinterFactory implements IPrinterFactory {
                     listAccessor.set(b, start, l);
                     arg.first = ps;
                     listAccessor.accept(printVisitor, arg);
-                } catch (Exception e) {
+                } catch (AsterixException e) {
                     throw new HyracksDataException(e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinter.java
deleted file mode 100644
index 41cf277..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinter.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.IOException;
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import org.apache.asterix.om.base.temporal.GregorianCalendarSystem;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.algebricks.data.utils.WriteValueTools;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class AYearMonthDurationPrinter implements IPrinter {
-
-    public static final AYearMonthDurationPrinter INSTANCE = new AYearMonthDurationPrinter();
-    private static final GregorianCalendarSystem gCalInstance = GregorianCalendarSystem.getInstance();
-
-    /* (non-Javadoc)
-     * @see org.apache.hyracks.algebricks.data.IPrinter#init()
-     */
-    @Override
-    public void init() {
-    }
-
-    /* (non-Javadoc)
-     * @see org.apache.hyracks.algebricks.data.IPrinter#print(byte[], int, int, java.io.PrintStream)
-     */
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        boolean positive = true;
-        int months = AInt32SerializerDeserializer.getInt(b, s + 1);
-
-        // set the negative flag. "||" is necessary in case that months field is not there (so it is 0)
-        if (months < 0) {
-            months *= -1;
-            positive = false;
-        }
-
-        int month = gCalInstance.getDurationMonth(months);
-        int year = gCalInstance.getDurationYear(months);
-
-        ps.print("year-month-duration(\"");
-        if (!positive) {
-            ps.print("-");
-        }
-        try {
-            ps.print("P");
-            if (year != 0) {
-                WriteValueTools.writeInt(year, ps);
-                ps.print("Y");
-            }
-            if (month != 0) {
-                WriteValueTools.writeInt(month, ps);
-                ps.print("M");
-            }
-            ps.print("\")");
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinterFactory.java
index ff16fb7..4f3782d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/AYearMonthDurationPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,12 +29,14 @@ public class AYearMonthDurationPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final AYearMonthDurationPrinterFactory INSTANCE = new AYearMonthDurationPrinterFactory();
 
-    /* (non-Javadoc)
-     * @see org.apache.hyracks.algebricks.data.IPrinterFactory#createPrinter()
-     */
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("year-month-duration(\"");
+        PrintTools.printYearMonthDurationString(b, s, l, ps);
+        ps.print("\")");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return AYearMonthDurationPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinter.java
deleted file mode 100644
index c373f53..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinter.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.adm;
-
-import java.io.IOException;
-import java.io.PrintStream;
-
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.algebricks.data.utils.WriteValueTools;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.primitive.ShortPointable;
-
-public class ShortWithoutTypeInfoPrinter implements IPrinter {
-
-    public static final ShortWithoutTypeInfoPrinter INSTANCE = new ShortWithoutTypeInfoPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        short d = ShortPointable.getShort(b, s);
-        try {
-            WriteValueTools.writeInt((int)d, ps);
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinterFactory.java
index 35005f8..666fa0a 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/adm/ShortWithoutTypeInfoPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.adm;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +29,11 @@ public class ShortWithoutTypeInfoPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ShortWithoutTypeInfoPrinterFactory INSTANCE = new ShortWithoutTypeInfoPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> ps
+            .print(ABooleanSerializerDeserializer.getBoolean(b, s + 1));
+
     @Override
     public IPrinter createPrinter() {
-        return ShortWithoutTypeInfoPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryHexPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryHexPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryHexPrinter.java
deleted file mode 100644
index 707ca70..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryHexPrinter.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.dataflow.data.nontagged.printers.csv;
-
-import java.io.IOException;
-import java.io.PrintStream;
-
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
-import org.apache.hyracks.util.bytes.HexPrinter;
-
-public class ABinaryHexPrinter implements IPrinter {
-    private ABinaryHexPrinter() {
-    }
-
-    public static final ABinaryHexPrinter INSTANCE = new ABinaryHexPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        int validLength = ByteArrayPointable.getContentLength(b, s + 1);
-        int start = s + 1 + ByteArrayPointable.getNumberBytesToStoreMeta(validLength);
-        try {
-            HexPrinter.printHexString(b, start, validLength, ps);
-        } catch (IOException e) {
-            throw new HyracksDataException(e);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryHexPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryHexPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryHexPrinterFactory.java
new file mode 100644
index 0000000..ad9de14
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryHexPrinterFactory.java
@@ -0,0 +1,53 @@
+/*
+ * 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.printers.csv;
+
+import java.io.IOException;
+import java.io.PrintStream;
+
+import org.apache.hyracks.algebricks.data.IPrinter;
+import org.apache.hyracks.algebricks.data.IPrinterFactory;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.primitive.ByteArrayPointable;
+import org.apache.hyracks.util.bytes.HexPrinter;
+
+public class ABinaryHexPrinterFactory implements IPrinterFactory {
+    private static final long serialVersionUID = 1L;
+
+    private ABinaryHexPrinterFactory() {
+    }
+
+    public static final ABinaryHexPrinterFactory INSTANCE = new ABinaryHexPrinterFactory();
+
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        int validLength = ByteArrayPointable.getContentLength(b, s + 1);
+        int start = s + 1 + ByteArrayPointable.getNumberBytesToStoreMeta(validLength);
+        try {
+            HexPrinter.printHexString(b, start, validLength, ps);
+        } catch (IOException e) {
+            throw new HyracksDataException(e);
+        }
+    };
+
+    @Override
+    public IPrinter createPrinter() {
+        return PRINTER;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryPrinterFactory.java
deleted file mode 100644
index 83a93fc..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABinaryPrinterFactory.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.asterix.dataflow.data.nontagged.printers.csv;
-
-import org.apache.asterix.dataflow.data.nontagged.printers.json.clean.ABinaryHexPrinter;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.algebricks.data.IPrinterFactory;
-
-public class ABinaryPrinterFactory implements IPrinterFactory {
-    private static final long serialVersionUID = 1L;
-
-    private ABinaryPrinterFactory() {
-    }
-
-    public static final ABinaryPrinterFactory INSTANCE = new ABinaryPrinterFactory();
-
-    @Override
-    public IPrinter createPrinter() {
-        return ABinaryHexPrinter.INSTANCE;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinter.java
deleted file mode 100644
index 01f50d5..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinter.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.csv;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ABooleanPrinter implements IPrinter {
-
-    public static final ABooleanPrinter INSTANCE = new ABooleanPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print(ABooleanSerializerDeserializer.getBoolean(b, s + 1));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinterFactory.java
index 43e7f8f..4aa6ccd 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ABooleanPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.csv;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ABooleanSerializerDeserializer;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +29,11 @@ public class ABooleanPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ABooleanPrinterFactory INSTANCE = new ABooleanPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> ps
+            .print(ABooleanSerializerDeserializer.getBoolean(b, s + 1));
+
     @Override
     public IPrinter createPrinter() {
-        return ABooleanPrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinter.java
deleted file mode 100644
index 91a9aa9..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinter.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.csv;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ACirclePrinter implements IPrinter {
-
-    public static final ACirclePrinter INSTANCE = new ACirclePrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("\"[ [");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
-        ps.print(", ");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 9));
-        ps.print("], ");
-        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 17));
-        ps.print(" ]\"");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinterFactory.java
index 630e80c..423a963 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ACirclePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.csv;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.ADoubleSerializerDeserializer;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +29,18 @@ public class ACirclePrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ACirclePrinterFactory INSTANCE = new ACirclePrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("\"[ [");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
+        ps.print(", ");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 9));
+        ps.print("], ");
+        ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 17));
+        ps.print(" ]\"");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return ACirclePrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinter.java
deleted file mode 100644
index e21f828..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinter.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.csv;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ADatePrinter implements IPrinter {
-
-    public static final ADatePrinter INSTANCE = new ADatePrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("\"");
-        PrintTools.printDateString(b, s, l, ps);
-        ps.print("\"");
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinterFactory.java
index 09ee227..98db505 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADatePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.csv;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
 import org.apache.hyracks.algebricks.data.IPrinter;
 import org.apache.hyracks.algebricks.data.IPrinterFactory;
 
@@ -26,9 +29,14 @@ public class ADatePrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ADatePrinterFactory INSTANCE = new ADatePrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("\"");
+        PrintTools.printDateString(b, s, l, ps);
+        ps.print("\"");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return ADatePrinter.INSTANCE;
+        return PRINTER;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-asterixdb/blob/fd7fb5a1/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADateTimePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADateTimePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADateTimePrinter.java
deleted file mode 100644
index c5d97d9..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/csv/ADateTimePrinter.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-package org.apache.asterix.dataflow.data.nontagged.printers.csv;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.printers.PrintTools;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class ADateTimePrinter implements IPrinter {
-
-    public static final ADateTimePrinter INSTANCE = new ADateTimePrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("\"");
-        PrintTools.printDateTimeString(b, s, l, ps);
-        ps.print("\"");
-    }
-}