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:14 UTC

[4/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/json/clean/AObjectPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
index c85e8f2..8ef4a83 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AObjectPrinterFactory.java
@@ -18,17 +18,145 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
+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.json.clean.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 CIRCLE:
+                ACirclePrinterFactory.PRINTER.print(b, s, l, ps);
+                return true;
+            case RECTANGLE:
+                ARectanglePrinterFactory.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;
+            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/json/clean/AOrderedlistPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AOrderedlistPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AOrderedlistPrinterFactory.java
index d911594..f517009 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AOrderedlistPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/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/json/clean/APoint3DPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APoint3DPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APoint3DPrinter.java
deleted file mode 100644
index 4271695..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/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.json.clean;
-
-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("[");
-        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/json/clean/APoint3DPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APoint3DPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APoint3DPrinterFactory.java
index 6bd43ae..84f6f16 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APoint3DPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APoint3DPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
+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("[");
+        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/json/clean/APointPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APointPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APointPrinter.java
deleted file mode 100644
index 1a6b327..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/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.json.clean;
-
-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("[");
-        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/json/clean/APointPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APointPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APointPrinterFactory.java
index 23cf4bf..b4f2979 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APointPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APointPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
+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("[");
+        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/json/clean/APolygonPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APolygonPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APolygonPrinter.java
deleted file mode 100644
index 5e8aea8..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APolygonPrinter.java
+++ /dev/null
@@ -1,58 +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.json.clean;
-
-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("[ ");
-
-        for (int i = 0; i < numberOfPoints; i++) {
-            if (i > 0)
-                ps.print(", ");
-
-            ps.print("[");
-            ps.print(ADoubleSerializerDeserializer.getDouble(b, s));
-            ps.print(", ");
-            ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 8));
-            ps.print("]");
-
-            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/json/clean/APolygonPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APolygonPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APolygonPrinterFactory.java
index b725feb..b97bd7b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APolygonPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/APolygonPrinterFactory.java
@@ -18,6 +18,10 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
+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,26 @@ 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("[ ");
+        for (int i = 0; i < numberOfPoints; i++) {
+            if (i > 0) {
+                ps.print(", ");
+            }
+            ps.print("[");
+            ps.print(ADoubleSerializerDeserializer.getDouble(b, s));
+            ps.print(", ");
+            ps.print(ADoubleSerializerDeserializer.getDouble(b, s + 8));
+            ps.print("]");
+            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/json/clean/ARecordPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARecordPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARecordPrinterFactory.java
index bf459e3..af2dff3 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARecordPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/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/json/clean/ARectanglePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARectanglePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARectanglePrinter.java
deleted file mode 100644
index 73a9e5d..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/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.json.clean;
-
-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("[ [");
-        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/json/clean/ARectanglePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARectanglePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARectanglePrinterFactory.java
index 5fcf690..2e66326 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARectanglePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ARectanglePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
+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("[ [");
+        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/json/clean/AStringPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AStringPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AStringPrinter.java
deleted file mode 100644
index de59073..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AStringPrinter.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.json.clean;
-
-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 {
-            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/json/clean/AStringPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AStringPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AStringPrinterFactory.java
index 3a57ae9..f06b743 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AStringPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AStringPrinterFactory.java
@@ -18,17 +18,29 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
+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 {
+            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/json/clean/ATimePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ATimePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ATimePrinter.java
deleted file mode 100644
index cda475d..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ATimePrinter.java
+++ /dev/null
@@ -1,42 +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.json.clean;
-
-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("\"");
-        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/json/clean/ATimePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ATimePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ATimePrinterFactory.java
index 11436ea..5729c33 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ATimePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/ATimePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
+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("\"");
+        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/json/clean/AUUIDPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinter.java
deleted file mode 100644
index 3990dd4..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/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.json.clean;
-
-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 + 2);
-        buf.append('"');
-        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/json/clean/AUUIDPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinterFactory.java
index eeda0e2..f1c853f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUUIDPrinterFactory.java
@@ -19,7 +19,9 @@
 
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
-import org.apache.asterix.dataflow.data.nontagged.printers.adm.AUUIDPrinter;
+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;
 
@@ -29,8 +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 + 2);
+        buf.append('"');
+        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/json/clean/AUnionPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUnionPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUnionPrinterFactory.java
index d19035e..9a2e630 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUnionPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/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/json/clean/AUnorderedlistPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUnorderedlistPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUnorderedlistPrinterFactory.java
index 3c4bcb5..a5c9d00 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUnorderedlistPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AUnorderedlistPrinterFactory.java
@@ -35,7 +35,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 +43,14 @@ public class AUnorderedlistPrinterFactory implements IPrinterFactory {
 
     @Override
     public IPrinter createPrinter() {
-
         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();

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/json/clean/AYearMonthDurationPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AYearMonthDurationPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AYearMonthDurationPrinter.java
deleted file mode 100644
index 2620afc..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AYearMonthDurationPrinter.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.json.clean;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class AYearMonthDurationPrinter implements IPrinter {
-
-    public static final AYearMonthDurationPrinter INSTANCE = new AYearMonthDurationPrinter();
-
-    /* (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 {
-
-        int months = AInt32SerializerDeserializer.getInt(b, s + 1);
-
-        ps.print("{ \"year-month-duration\": ");
-        ps.print(months);
-        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/json/clean/AYearMonthDurationPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AYearMonthDurationPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AYearMonthDurationPrinterFactory.java
index 16886db..81abdc0 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AYearMonthDurationPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/clean/AYearMonthDurationPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.clean;
 
+import java.io.PrintStream;
+
+import org.apache.asterix.dataflow.data.nontagged.serde.AInt32SerializerDeserializer;
 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\": ");
+        ps.print(AInt32SerializerDeserializer.getInt(b, s + 1));
+        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/json/lossless/ABinaryHexPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryHexPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryHexPrinter.java
deleted file mode 100644
index 74f3353..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryHexPrinter.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.json.lossless;
-
-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 {
-            ps.print("\"");
-            HexPrinter.printHexString(b, start, validLength, ps);
-            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/json/lossless/ABinaryHexPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryHexPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryHexPrinterFactory.java
new file mode 100644
index 0000000..153574c
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryHexPrinterFactory.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.printers.json.lossless;
+
+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;
+
+    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 {
+            ps.print("\"");
+            HexPrinter.printHexString(b, start, validLength, ps);
+            ps.print("\"");
+        } 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/json/lossless/ABinaryPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryPrinterFactory.java
deleted file mode 100644
index 65c4fad..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABinaryPrinterFactory.java
+++ /dev/null
@@ -1,37 +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.json.lossless;
-
-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/json/lossless/ABooleanPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABooleanPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABooleanPrinter.java
deleted file mode 100644
index aa94664..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/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.json.lossless;
-
-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/json/lossless/ABooleanPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABooleanPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABooleanPrinterFactory.java
index 8315212..b65897b 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABooleanPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ABooleanPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.lossless;
 
+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/json/lossless/ACirclePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ACirclePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ACirclePrinter.java
deleted file mode 100644
index eb42de6..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/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.json.lossless;
-
-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("{ \"circle\": [ { \"point\": [");
-        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/json/lossless/ACirclePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ACirclePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ACirclePrinterFactory.java
index 83cf09d..5ec854d 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ACirclePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ACirclePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.lossless;
 
+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("{ \"circle\": [ { \"point\": [");
+        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/json/lossless/ADatePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADatePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADatePrinter.java
deleted file mode 100644
index c9a2beb..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/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.json.lossless;
-
-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("{ \"date\": \"");
-        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/json/lossless/ADatePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADatePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADatePrinterFactory.java
index 55c4493..4a43fbc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADatePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADatePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.lossless;
 
+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("{ \"date\": \"");
+        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/json/lossless/ADateTimePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADateTimePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADateTimePrinter.java
deleted file mode 100644
index 78f75d8..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/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.json.lossless;
-
-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("{ \"datetime\": \"");
-        PrintTools.printDateTimeString(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/json/lossless/ADateTimePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADateTimePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADateTimePrinterFactory.java
index af12e42..cd5ae8f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADateTimePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADateTimePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.lossless;
 
+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 ADateTimePrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ADateTimePrinterFactory INSTANCE = new ADateTimePrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("{ \"datetime\": \"");
+        PrintTools.printDateTimeString(b, s, l, ps);
+        ps.print("\" }");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return ADateTimePrinter.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/json/lossless/ADayTimeDurationPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADayTimeDurationPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADayTimeDurationPrinter.java
deleted file mode 100644
index 4730cd9..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADayTimeDurationPrinter.java
+++ /dev/null
@@ -1,42 +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.json.lossless;
-
-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 ADayTimeDurationPrinter implements IPrinter {
-
-    public static final ADayTimeDurationPrinter INSTANCE = new ADayTimeDurationPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("{ \"day-time-duration\": ");
-        PrintTools.printDayTimeDurationString(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/json/lossless/ADayTimeDurationPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADayTimeDurationPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADayTimeDurationPrinterFactory.java
index e28f3aa..028ed20 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADayTimeDurationPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADayTimeDurationPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.lossless;
 
+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 ADayTimeDurationPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ADayTimeDurationPrinterFactory INSTANCE = new ADayTimeDurationPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("{ \"day-time-duration\": ");
+        PrintTools.printDayTimeDurationString(b, s, l, ps);
+        ps.print("}");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return ADayTimeDurationPrinter.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/json/lossless/ADoublePrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADoublePrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADoublePrinter.java
deleted file mode 100644
index 3175670..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADoublePrinter.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.json.lossless;
-
-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 ADoublePrinter implements IPrinter {
-
-    public static final ADoublePrinter INSTANCE = new ADoublePrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print(ADoubleSerializerDeserializer.getDouble(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/json/lossless/ADoublePrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADoublePrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADoublePrinterFactory.java
index 7bb3461..a14a16f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADoublePrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADoublePrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.lossless;
 
+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,11 @@ public class ADoublePrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ADoublePrinterFactory INSTANCE = new ADoublePrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> ps
+            .print(ADoubleSerializerDeserializer.getDouble(b, s + 1));
+
     @Override
     public IPrinter createPrinter() {
-        return ADoublePrinter.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/json/lossless/ADurationPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADurationPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADurationPrinter.java
deleted file mode 100644
index 3b5a1a2..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADurationPrinter.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.json.lossless;
-
-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 ADurationPrinter implements IPrinter {
-
-    public static final ADurationPrinter INSTANCE = new ADurationPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print("{ \"duration\": \"");
-        PrintTools.printDurationString(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/json/lossless/ADurationPrinterFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADurationPrinterFactory.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADurationPrinterFactory.java
index 29d0388..a4844fc 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADurationPrinterFactory.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/ADurationPrinterFactory.java
@@ -18,6 +18,9 @@
  */
 package org.apache.asterix.dataflow.data.nontagged.printers.json.lossless;
 
+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 ADurationPrinterFactory implements IPrinterFactory {
     private static final long serialVersionUID = 1L;
     public static final ADurationPrinterFactory INSTANCE = new ADurationPrinterFactory();
 
+    public static final IPrinter PRINTER = (byte[] b, int s, int l, PrintStream ps) -> {
+        ps.print("{ \"duration\": \"");
+        PrintTools.printDurationString(b, s, l, ps);
+        ps.print("\" }");
+    };
+
     @Override
     public IPrinter createPrinter() {
-        return ADurationPrinter.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/json/lossless/AFloatPrinter.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/AFloatPrinter.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/AFloatPrinter.java
deleted file mode 100644
index ddbe830..0000000
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/dataflow/data/nontagged/printers/json/lossless/AFloatPrinter.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.json.lossless;
-
-import java.io.PrintStream;
-
-import org.apache.asterix.dataflow.data.nontagged.serde.AFloatSerializerDeserializer;
-import org.apache.hyracks.algebricks.data.IPrinter;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-
-public class AFloatPrinter implements IPrinter {
-
-    public static final AFloatPrinter INSTANCE = new AFloatPrinter();
-
-    @Override
-    public void init() {
-    }
-
-    @Override
-    public void print(byte[] b, int s, int l, PrintStream ps) throws HyracksDataException {
-        ps.print(AFloatSerializerDeserializer.getFloat(b, s + 1));
-    }
-}