You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Murtadha Hubail (Code Review)" <de...@asterixdb.apache.org> on 2019/05/27 00:34:47 UTC

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Murtadha Hubail has uploaded this change for review. ( https://asterix-gerrit.ics.uci.edu/3410


Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................

[NO ISSUE][OTH] Introduce ResponsePrinter

- user model changes: no
- storage format changes: no
- interface changes: yes

Details:
- Introduce ResponsePrinter that is responsible for printing
  the returned response from query service.
- Make ResponsePrinter responsible for adding separators between
  fields in the returned response.
- Introduce IResponseFieldPrinter and encapsulate each response
  field in its own printer.
- Introduce ICodedMessage and use it for any object that
  has code<->message (e.g errors and warnings).

Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java
C asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java
C asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
C asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
C asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java
38 files changed, 1,273 insertions(+), 407 deletions(-)



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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
index b244c0c..26ebbd5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
@@ -20,6 +20,7 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -43,8 +44,10 @@
      *            provides query language related components
      * @param storageComponentProvider
      *            provides storage related components
+     * @param responsePrinter
      * @return an implementation of {@code IStatementExecutor} thaxt is used to execute the passed list of statements
      */
     IStatementExecutor create(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
-            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider);
+            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider,
+            IResponsePrinter responsePrinter);
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
index 97125f0..ed59cf4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
@@ -38,6 +38,10 @@
     private SessionOutput.ResultAppender handleAppender;
     private final SessionOutput.ResultAppender statusAppender;
 
+    public SessionOutput(PrintWriter out) {
+        this(null, out);
+    }
+
     public SessionOutput(SessionConfig config, PrintWriter out) {
         this(config, out, null, null, null, null);
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
index 6f27eb5..4dadf55 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
@@ -20,7 +20,6 @@
 
 import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_CONNECTION_ATTR;
 
-import java.io.PrintWriter;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.common.api.IApplicationContext;
@@ -38,30 +37,6 @@
     private static final Logger LOGGER = LogManager.getLogger();
     protected final IApplicationContext appCtx;
 
-    public enum ResultFields {
-        REQUEST_ID("requestID"),
-        CLIENT_ID("clientContextID"),
-        SIGNATURE("signature"),
-        TYPE("type"),
-        STATUS("status"),
-        RESULTS("results"),
-        HANDLE("handle"),
-        ERRORS("errors"),
-        METRICS("metrics"),
-        PLANS("plans"),
-        WARNINGS("warnings");
-
-        private final String str;
-
-        ResultFields(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
     public enum ResultStatus {
         RUNNING("running"),
         SUCCESS("success"),
@@ -72,22 +47,6 @@
         private final String str;
 
         ResultStatus(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    public enum ErrorField {
-        CODE("code"),
-        MSG("msg"),
-        STACK("stack");
-
-        private final String str;
-
-        ErrorField(String str) {
             this.str = str;
         }
 
@@ -118,13 +77,4 @@
         }
         return hcc;
     }
-
-    protected static void printRequestId(PrintWriter pw, String requestId) {
-        ResultUtil.printField(pw, ResultFields.REQUEST_ID.str(), requestId);
-    }
-
-    protected static void printHandle(PrintWriter pw, String handle, boolean comma) {
-        ResultUtil.printField(pw, ResultFields.HANDLE.str(), handle, comma);
-    }
-
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index ea6e616..f395931 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -32,6 +32,7 @@
 
 import javax.imageio.ImageIO;
 
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.api.IRequestReference;
 import org.apache.asterix.common.config.GlobalConfig;
@@ -147,7 +148,7 @@
             SessionOutput sessionOutput = new SessionOutput(sessionConfig, out);
             MetadataManager.INSTANCE.init();
             IStatementExecutor translator = statementExectorFactory.create(appCtx, statements, sessionOutput,
-                    compilationProvider, componentProvider);
+                    compilationProvider, componentProvider, new ResponsePrinter(sessionOutput));
             double duration;
             long startTime = System.currentTimeMillis();
             final IRequestParameters requestParameters = new RequestParameters(requestReference, query, resultSet,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index 8923ae8..0ab014f 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -29,7 +29,9 @@
 import org.apache.asterix.app.message.CancelQueryRequest;
 import org.apache.asterix.app.message.ExecuteStatementRequestMessage;
 import org.apache.asterix.app.message.ExecuteStatementResponseMessage;
-import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResponsePrinter;
+import org.apache.asterix.app.result.fields.NcResultPrinter;
+import org.apache.asterix.app.result.fields.PlansPrinter;
 import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.IRequestReference;
@@ -40,14 +42,10 @@
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.MessageFuture;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.ResultProperties;
 import org.apache.asterix.translator.SessionOutput;
-import org.apache.commons.lang3.tuple.Triple;
 import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.http.api.IChannelClosedHandler;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.server.HttpServer;
@@ -73,7 +71,8 @@
     protected void executeStatement(IRequestReference requestReference, String statementsText,
             SessionOutput sessionOutput, ResultProperties resultProperties, IStatementExecutor.Stats stats,
             QueryServiceRequestParameters param, RequestExecutionState execution,
-            Map<String, String> optionalParameters, Map<String, byte[]> statementParameters) throws Exception {
+            Map<String, String> optionalParameters, Map<String, byte[]> statementParameters,
+            ResponsePrinter responseWriter) throws Exception {
         // Running on NC -> send 'execute' message to CC
         INCServiceContext ncCtx = (INCServiceContext) serviceCtx;
         INCMessageBroker ncMb = (INCMessageBroker) ncCtx.getMessageBroker();
@@ -119,20 +118,12 @@
                 throw new Exception(err.toString(), err);
             }
         }
-        // no errors - stop buffering and allow for streaming result delivery
-        sessionOutput.release();
-
-        IStatementExecutor.ResultMetadata resultMetadata = responseMsg.getMetadata();
-        if (delivery == IStatementExecutor.ResultDelivery.IMMEDIATE && !resultMetadata.getResultSets().isEmpty()) {
-            stats.setProcessedObjects(responseMsg.getStats().getProcessedObjects());
-            for (Triple<JobId, ResultSetId, ARecordType> rsmd : resultMetadata.getResultSets()) {
-                ResultReader resultReader = new ResultReader(getResultSet(), rsmd.getLeft(), rsmd.getMiddle());
-                ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, rsmd.getRight());
-            }
-        } else {
-            sessionOutput.out().append(responseMsg.getResult());
+        if (hasResult(responseMsg)) {
+            responseWriter.addResultPrinter(
+                    new NcResultPrinter(appCtx, responseMsg, getResultSet(), delivery, sessionOutput));
         }
-        printExecutionPlans(sessionOutput, responseMsg.getExecutionPlans());
+        responseWriter.addResultPrinter(
+                new PlansPrinter(responseMsg.getExecutionPlans(), sessionOutput.config().getPlanFormat()));
     }
 
     private void cancelQuery(INCMessageBroker messageBroker, String nodeId, String uuid, String clientContextID,
@@ -174,4 +165,8 @@
     public IChannelClosedHandler getChannelClosedHandler(HttpServer server) {
         return InterruptOnCloseHandler.INSTANCE;
     }
+
+    private static boolean hasResult(ExecuteStatementResponseMessage responseMsg) {
+        return !responseMsg.getMetadata().getResultSets().isEmpty() || !responseMsg.getResult().isEmpty();
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index cda4d34..2b06dbb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -22,8 +22,12 @@
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.api.common.ResultMetadata;
+import org.apache.asterix.app.result.ResponseMertics;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.fields.MetricsPrinter;
+import org.apache.asterix.app.result.fields.ResultsPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
@@ -87,15 +91,19 @@
             }
             ResultMetadata metadata = (ResultMetadata) resultReader.getMetadata();
             SessionOutput sessionOutput = initResponse(request, response, metadata.getFormat());
+            ResponsePrinter printer = new ResponsePrinter(sessionOutput);
             if (metadata.getFormat() == SessionConfig.OutputFormat.CLEAN_JSON
                     || metadata.getFormat() == SessionConfig.OutputFormat.LOSSLESS_JSON) {
                 final Stats stats = new Stats();
-                sessionOutput.out().print("{\n");
-                ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, null);
-                QueryServiceServlet.printMetrics(sessionOutput.out(), System.nanoTime() - elapsedStart,
-                        metadata.getJobDuration(), stats.getCount(), stats.getSize(), metadata.getProcessedObjects(), 0,
-                        0, HttpUtil.getPreferredCharset(request));
-                sessionOutput.out().print("}\n");
+                printer.begin();
+                printer.addResultPrinter(new ResultsPrinter(appCtx, resultReader, null, stats, sessionOutput));
+                printer.printResults();
+                ResponseMertics mertics =
+                        ResponseMertics.of(System.nanoTime() - elapsedStart, metadata.getJobDuration(),
+                                stats.getCount(), stats.getSize(), metadata.getProcessedObjects(), 0, 0);
+                printer.addFooterPrinter(new MetricsPrinter(mertics, HttpUtil.getPreferredCharset(request)));
+                printer.printFooters();
+                printer.end();
             } else {
                 ResultUtil.printResults(appCtx, resultReader, sessionOutput, new Stats(), null);
             }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index 06b75e3..83b8f63 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -31,7 +31,6 @@
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -43,10 +42,23 @@
 import java.util.function.Function;
 
 import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.app.result.ExecutionError;
+import org.apache.asterix.app.result.ResponseMertics;
+import org.apache.asterix.app.result.ResponsePrinter;
+import org.apache.asterix.app.result.fields.ClientContextIdPrinter;
+import org.apache.asterix.app.result.fields.ErrorsPrinter;
+import org.apache.asterix.app.result.fields.MetricsPrinter;
+import org.apache.asterix.app.result.fields.ParseOnlyResultPrinter;
+import org.apache.asterix.app.result.fields.PlansPrinter;
+import org.apache.asterix.app.result.fields.RequestIdPrinter;
+import org.apache.asterix.app.result.fields.SignaturePrinter;
+import org.apache.asterix.app.result.fields.StatusPrinter;
+import org.apache.asterix.app.result.fields.TypePrinter;
+import org.apache.asterix.app.result.fields.WarningsPrinter;
 import org.apache.asterix.app.translator.QueryTranslator;
-import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.api.ICodedMessage;
 import org.apache.asterix.common.api.IReceptionist;
 import org.apache.asterix.common.api.IRequestReference;
 import org.apache.asterix.common.config.GlobalConfig;
@@ -64,8 +76,6 @@
 import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.om.base.IAObject;
-import org.apache.asterix.translator.ExecutionPlans;
-import org.apache.asterix.translator.ExecutionPlansJsonPrintUtil;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutor.ResultDelivery;
@@ -184,26 +194,6 @@
         }
     }
 
-    private enum Metrics {
-        ELAPSED_TIME("elapsedTime"),
-        EXECUTION_TIME("executionTime"),
-        RESULT_COUNT("resultCount"),
-        RESULT_SIZE("resultSize"),
-        ERROR_COUNT("errorCount"),
-        PROCESSED_OBJECTS_COUNT("processedObjects"),
-        WARNING_COUNT("warningCount");
-
-        private final String str;
-
-        Metrics(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
     protected static final class RequestExecutionState {
         private long execStart = -1;
         private long execEnd = -1;
@@ -288,68 +278,6 @@
         SessionOutput.ResultAppender appendStatus = ResultUtil.createResultStatusAppender();
         SessionConfig sessionConfig = new SessionConfig(SessionConfig.OutputFormat.CLEAN_JSON);
         return new SessionOutput(sessionConfig, resultWriter, resultPrefix, resultPostfix, null, appendStatus);
-    }
-
-    private static void printClientContextID(PrintWriter pw, QueryServiceRequestParameters params) {
-        if (params.getClientContextID() != null && !params.getClientContextID().isEmpty()) {
-            ResultUtil.printField(pw, ResultFields.CLIENT_ID.str(), params.getClientContextID());
-        }
-    }
-
-    private static void printSignature(PrintWriter pw, QueryServiceRequestParameters param) {
-        if (param.isSignature()) {
-            pw.print("\t\"");
-            pw.print(ResultFields.SIGNATURE.str());
-            pw.print("\": {\n");
-            pw.print("\t");
-            ResultUtil.printField(pw, "*", "*", false);
-            pw.print("\t},\n");
-        }
-    }
-
-    private static void printType(PrintWriter pw, SessionConfig sessionConfig) {
-        switch (sessionConfig.fmt()) {
-            case ADM:
-                ResultUtil.printField(pw, ResultFields.TYPE.str(), HttpUtil.ContentType.APPLICATION_ADM);
-                break;
-            case CSV:
-                String contentType = HttpUtil.ContentType.CSV + "; header="
-                        + (sessionConfig.is(SessionConfig.FORMAT_CSV_HEADER) ? "present" : "absent");
-                ResultUtil.printField(pw, ResultFields.TYPE.str(), contentType);
-                break;
-            default:
-                break;
-        }
-    }
-
-    public static void printMetrics(PrintWriter pw, long elapsedTime, long executionTime, long resultCount,
-            long resultSize, long processedObjects, long errorCount, long warnCount, Charset resultCharset) {
-        boolean hasErrors = errorCount != 0;
-        boolean hasWarnings = warnCount != 0;
-        boolean useAscii = !StandardCharsets.UTF_8.equals(resultCharset)
-                && !"μ".contentEquals(resultCharset.decode(resultCharset.encode("μ")));
-        pw.print("\t\"");
-        pw.print(ResultFields.METRICS.str());
-        pw.print("\": {\n");
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.ELAPSED_TIME.str(), Duration.formatNanos(elapsedTime, useAscii));
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.EXECUTION_TIME.str(), Duration.formatNanos(executionTime, useAscii));
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.RESULT_COUNT.str(), resultCount, true);
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.RESULT_SIZE.str(), resultSize, true);
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.PROCESSED_OBJECTS_COUNT.str(), processedObjects, hasWarnings || hasErrors);
-        if (hasWarnings) {
-            pw.print("\t");
-            ResultUtil.printField(pw, Metrics.WARNING_COUNT.str(), warnCount, hasErrors);
-        }
-        if (hasErrors) {
-            pw.print("\t");
-            ResultUtil.printField(pw, Metrics.ERROR_COUNT.str(), errorCount, false);
-        }
-        pw.print("\t}\n");
     }
 
     protected String getOptText(JsonNode node, Parameter parameter) {
@@ -539,72 +467,89 @@
         long errorCount = 1;
         Stats stats = new Stats();
         RequestExecutionState execution = new RequestExecutionState();
-        List<ExecutionWarning> warnings = Collections.emptyList();
+        List<ICodedMessage> warnings = Collections.emptyList();
         Charset resultCharset = HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
         PrintWriter httpWriter = response.writer();
         SessionOutput sessionOutput = createSessionOutput(httpWriter);
         QueryServiceRequestParameters param = newRequestParameters();
+        ResponsePrinter responsePrinter = new ResponsePrinter(sessionOutput);
+        ResultDelivery delivery = ResultDelivery.IMMEDIATE;
         try {
             // buffer the output until we are ready to set the status of the response message correctly
-            sessionOutput.hold();
-            sessionOutput.out().print("{\n");
+            responsePrinter.begin();
             Map<String, String> optionalParams = null;
             if (optionalParamProvider != null) {
                 optionalParams = optionalParamProvider.apply(request);
             }
             setRequestParam(request, param, optionalParams);
             LOGGER.info(() -> "handleRequest: " + LogRedactionUtil.userData(param.toString()));
-            ResultDelivery delivery = parseResultDelivery(param.getMode());
+            delivery = parseResultDelivery(param.getMode());
             setSessionConfig(sessionOutput, param, delivery);
             final ResultProperties resultProperties = param.getMaxResultReads() == null ? new ResultProperties(delivery)
                     : new ResultProperties(delivery, Long.parseLong(param.getMaxResultReads()));
-            printAdditionalResultFields(sessionOutput.out());
-            printRequestId(sessionOutput.out(), requestRef.getUuid());
-            printClientContextID(sessionOutput.out(), param);
-            if (!param.isParseOnly()) {
-                printSignature(sessionOutput.out(), param);
-            }
-            printType(sessionOutput.out(), sessionOutput.config());
+            buildResponseHeaders(requestRef, sessionOutput, param, responsePrinter, delivery);
+            responsePrinter.printHeaders();
             validateStatement(param.getStatement());
             String statementsText = param.getStatement() + ";";
             if (param.isParseOnly()) {
                 ResultUtil.ParseOnlyResult parseOnlyResult = parseStatement(statementsText);
                 setAccessControlHeaders(request, response);
-                response.setStatus(HttpResponseStatus.OK);
-                printParseOnlyValueResult(sessionOutput, parseOnlyResult);
-                ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
+                response.setStatus(execution.getHttpStatus());
+                responsePrinter.addResultPrinter(new ParseOnlyResultPrinter(parseOnlyResult));
             } else {
                 Map<String, byte[]> statementParams = org.apache.asterix.app.translator.RequestParameters
                         .serializeParameterValues(param.getStatementParams());
                 setAccessControlHeaders(request, response);
                 response.setStatus(execution.getHttpStatus());
                 executeStatement(requestRef, statementsText, sessionOutput, resultProperties, stats, param, execution,
-                        optionalParams, statementParams);
-                if (ResultDelivery.IMMEDIATE == delivery || ResultDelivery.DEFERRED == delivery) {
-                    ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
-                }
-            }
-            if (!warnings.isEmpty()) {
-                printWarnings(sessionOutput.out(), warnings);
+                        optionalParams, statementParams, responsePrinter);
             }
             errorCount = 0;
         } catch (Exception | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError e) {
             handleExecuteStatementException(e, execution, param);
             response.setStatus(execution.getHttpStatus());
-            printError(sessionOutput.out(), e);
-            ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
+            requestFailed(e, responsePrinter);
         } finally {
-            // make sure that we stop buffering and return the result to the http response
-            sessionOutput.release();
             execution.finish();
         }
-        printMetrics(sessionOutput.out(), System.nanoTime() - elapsedStart, execution.duration(), stats.getCount(),
-                stats.getSize(), stats.getProcessedObjects(), errorCount, warnings.size(), resultCharset);
-        sessionOutput.out().print("}\n");
-        sessionOutput.out().flush();
+        responsePrinter.printResults();
+        buildResponseFooters(elapsedStart, errorCount, stats, execution, warnings, resultCharset, responsePrinter,
+                delivery);
+        responsePrinter.printFooters();
+        responsePrinter.end();
         if (sessionOutput.out().checkError()) {
             LOGGER.warn("Error flushing output writer");
         }
+    }
+
+    protected void buildResponseHeaders(IRequestReference requestRef, SessionOutput sessionOutput,
+            QueryServiceRequestParameters param, ResponsePrinter responsePrinter, ResultDelivery delivery) {
+        responsePrinter.addHeaderPrinter(new RequestIdPrinter(requestRef.getUuid()));
+        if (param.getClientContextID() != null && !param.getClientContextID().isEmpty()) {
+            responsePrinter.addHeaderPrinter(new ClientContextIdPrinter(param.getClientContextID()));
+        }
+        if (param.isSignature() && delivery != ResultDelivery.ASYNC && !param.isParseOnly()) {
+            responsePrinter.addHeaderPrinter(new SignaturePrinter());
+        }
+        if (sessionOutput.config().fmt() == SessionConfig.OutputFormat.ADM
+                || sessionOutput.config().fmt() == SessionConfig.OutputFormat.CSV) {
+            responsePrinter.addHeaderPrinter(new TypePrinter(sessionOutput.config()));
+        }
+    }
+
+    private void buildResponseFooters(long elapsedStart, long errorCount, Stats stats, RequestExecutionState execution,
+            List<ICodedMessage> warnings, Charset resultCharset, ResponsePrinter responsePrinter,
+            ResultDelivery delivery) {
+        if (ResultDelivery.ASYNC != delivery) {
+            // in case of ASYNC delivery, the status is printed by query translator
+            responsePrinter.addFooterPrinter(new StatusPrinter(execution.getResultStatus()));
+        }
+        if (!warnings.isEmpty()) {
+            responsePrinter.addFooterPrinter(new WarningsPrinter(warnings));
+        }
+        final ResponseMertics mertics = ResponseMertics.of(System.nanoTime() - elapsedStart, execution.duration(),
+                stats.getCount(), stats.getSize(), stats.getProcessedObjects(), errorCount, warnings.size());
+        responsePrinter.addFooterPrinter(new MetricsPrinter(mertics, resultCharset));
     }
 
     protected void validateStatement(String statement) throws RuntimeDataException {
@@ -628,7 +573,8 @@
     protected void executeStatement(IRequestReference requestReference, String statementsText,
             SessionOutput sessionOutput, ResultProperties resultProperties, Stats stats,
             QueryServiceRequestParameters param, RequestExecutionState execution,
-            Map<String, String> optionalParameters, Map<String, byte[]> statementParameters) throws Exception {
+            Map<String, String> optionalParameters, Map<String, byte[]> statementParameters,
+            ResponsePrinter responseWriter) throws Exception {
         IClusterManagementWork.ClusterState clusterState =
                 ((ICcApplicationContext) appCtx).getClusterStateManager().getState();
         if (clusterState != IClusterManagementWork.ClusterState.ACTIVE) {
@@ -639,7 +585,7 @@
         List<Statement> statements = parser.parse();
         MetadataManager.INSTANCE.init();
         IStatementExecutor translator = statementExecutorFactory.create((ICcApplicationContext) appCtx, statements,
-                sessionOutput, compilationProvider, componentProvider);
+                sessionOutput, compilationProvider, componentProvider, responseWriter);
         execution.start();
         Map<String, IAObject> stmtParams =
                 org.apache.asterix.app.translator.RequestParameters.deserializeParameterValues(statementParameters);
@@ -648,7 +594,8 @@
                 optionalParameters, stmtParams, param.isMultiStatement());
         translator.compileAndExecute(getHyracksClientConnection(), requestParameters);
         execution.end();
-        printExecutionPlans(sessionOutput, translator.getExecutionPlans());
+        responseWriter.addResultPrinter(
+                new PlansPrinter(translator.getExecutionPlans(), sessionOutput.config().getPlanFormat()));
     }
 
     protected void handleExecuteStatementException(Throwable t, RequestExecutionState state,
@@ -716,42 +663,9 @@
                 && "present".equals(getParameterValue(param.getFormat(), Attribute.HEADER.str())));
     }
 
-    protected void printError(PrintWriter sessionOut, Throwable throwable) {
-        ResultUtil.printError(sessionOut, throwable);
-    }
-
-    protected void printAdditionalResultFields(PrintWriter sessionOut) {
-        // do nothing
-    }
-
-    protected void printWarnings(PrintWriter pw, List<ExecutionWarning> warnings) {
-        ResultUtil.printWarnings(pw, warnings);
-    }
-
-    protected void printParseOnlyValueResult(SessionOutput output, ResultUtil.ParseOnlyResult parseOnlyResult) {
-        final PrintWriter pw = output.out();
-        pw.print("\t\"");
-        pw.print(ResultFields.RESULTS.str()); //TODO: use ResultUtil, ResultPrinter
-        pw.print("\":");
-        pw.print(parseOnlyResult.asJson());
-        pw.print(",\n");
-    }
-
-    protected void printExecutionPlans(SessionOutput output, ExecutionPlans executionPlans) {
-        final PrintWriter pw = output.out();
-        pw.print("\t\"");
-        pw.print(ResultFields.PLANS.str());
-        pw.print("\":");
-        final SessionConfig.PlanFormat planFormat = output.config().getPlanFormat();
-        switch (planFormat) {
-            case JSON:
-            case STRING:
-                pw.print(ExecutionPlansJsonPrintUtil.asJson(executionPlans, planFormat));
-                break;
-            default:
-                throw new IllegalStateException("Unrecognized plan format: " + planFormat);
-        }
-        pw.print(",\n");
+    protected void requestFailed(Throwable throwable, ResponsePrinter responsePrinter) {
+        final ExecutionError executionError = ExecutionError.of(throwable);
+        responsePrinter.addResultPrinter(new ErrorsPrinter(Collections.singletonList(executionError)));
     }
 
     protected QueryServiceRequestParameters newRequestParameters() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
index df09aee..c91f0e3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
@@ -21,12 +21,19 @@
 import static org.apache.asterix.api.http.server.AbstractQueryApiServlet.ResultStatus.FAILED;
 
 import java.io.PrintWriter;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.asterix.app.result.ExecutionError;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.fields.ErrorsPrinter;
+import org.apache.asterix.app.result.fields.ResultHandlePrinter;
+import org.apache.asterix.app.result.fields.StatusPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.translator.SessionOutput;
 import org.apache.hyracks.api.result.ResultJobRecord;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
@@ -52,37 +59,32 @@
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
         }
-
         ResultReader resultReader = new ResultReader(getResultSet(), handle.getJobId(), handle.getResultSetId());
-
         final ResultJobRecord.Status resultReaderStatus = resultReader.getStatus();
         if (resultReaderStatus == null) {
             LOGGER.log(Level.INFO, "No results for: \"" + strHandle + "\"");
             response.setStatus(HttpResponseStatus.NOT_FOUND);
             return;
         }
-
         ResultStatus resultStatus = resultStatus(resultReaderStatus);
         Exception ex = extractException(resultReaderStatus);
-
         HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
         final PrintWriter resultWriter = response.writer();
-
-        HttpResponseStatus httpStatus = HttpResponseStatus.OK;
-
-        resultWriter.print("{\n");
-        ResultUtil.printStatus(resultWriter, resultStatus, (ex != null) || ResultStatus.SUCCESS == resultStatus);
-
+        response.setStatus(HttpResponseStatus.OK);
+        SessionOutput sessionOutput = new SessionOutput(resultWriter);
+        ResponsePrinter printer = new ResponsePrinter(sessionOutput);
+        printer.begin();
+        printer.addHeaderPrinter(new StatusPrinter(resultStatus));
+        printer.printHeaders();
         if (ResultStatus.SUCCESS == resultStatus) {
             String servletPath = servletPath(request).replace("status", "result");
             String resHandle = "http://" + host(request) + servletPath + strHandle;
-            printHandle(resultWriter, resHandle, false);
+            printer.addResultPrinter(new ResultHandlePrinter(resHandle));
         } else if (ex != null) {
-            ResultUtil.printError(resultWriter, ex, false);
+            printer.addResultPrinter(new ErrorsPrinter(Collections.singletonList(ExecutionError.of(ex))));
         }
-
-        resultWriter.print("}\n");
-        response.setStatus(httpStatus);
+        printer.printResults();
+        printer.end();
         if (response.writer().checkError()) {
             LOGGER.warn("Error flushing output writer");
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
index fa3c03d..63da9ce 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
@@ -32,11 +32,12 @@
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultPrinter;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.fields.ResultHandlePrinter;
+import org.apache.asterix.app.result.fields.ResultsPrinter;
+import org.apache.asterix.app.result.fields.StatusPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.lang.aql.parser.TokenMgrError;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.om.types.ARecordType;
@@ -47,8 +48,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.util.JSONUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -91,76 +90,6 @@
         new ResultPrinter(appCtx, output, stats, recordType).print(record);
     }
 
-    public static void printResultHandle(SessionOutput output, ResultHandle handle) throws HyracksDataException {
-        try {
-            final AlgebricksAppendable app = new AlgebricksAppendable(output.out());
-            output.appendHandle(app, handle.toString());
-        } catch (AlgebricksException e) {
-            LOGGER.warn("error printing handle", e);
-        }
-    }
-
-    public static void printStatus(SessionOutput output, AbstractQueryApiServlet.ResultStatus rs) {
-        try {
-            final AlgebricksAppendable app = new AlgebricksAppendable(output.out());
-            output.appendStatus(app, rs.str());
-        } catch (AlgebricksException e) {
-            LOGGER.warn("error printing status", e);
-        }
-    }
-
-    public static void printStatus(PrintWriter pw, AbstractQueryApiServlet.ResultStatus rs, boolean comma) {
-        printField(pw, AbstractQueryApiServlet.ResultFields.STATUS.str(), rs.str(), comma);
-    }
-
-    public static void printError(PrintWriter pw, Throwable e) {
-        printError(pw, e, true);
-    }
-
-    public static void printError(PrintWriter pw, Throwable e, boolean comma) {
-        printError(pw, e, 1, comma);
-    }
-
-    public static void printError(PrintWriter pw, Throwable e, int code, boolean comma) {
-        Throwable rootCause = getRootCause(e);
-        String msg = rootCause.getMessage();
-        if (!(rootCause instanceof AlgebricksException || rootCause instanceof HyracksException
-                || rootCause instanceof TokenMgrError
-                || rootCause instanceof org.apache.asterix.aqlplus.parser.TokenMgrError)) {
-            msg = rootCause.getClass().getSimpleName() + (msg == null ? "" : ": " + msg);
-        }
-        printError(pw, msg, code, comma);
-    }
-
-    public static void printError(PrintWriter pw, String msg, int code, boolean comma) {
-        pw.print("\t\"");
-        pw.print(AbstractQueryApiServlet.ResultFields.ERRORS.str());
-        pw.print("\": [{ \n\t");
-        printField(pw, QueryServiceServlet.ErrorField.CODE.str(), code);
-        pw.print("\t");
-        printField(pw, QueryServiceServlet.ErrorField.MSG.str(), JSONUtil.escape(msg), false);
-        pw.print(comma ? "\t}],\n" : "\t}]\n");
-    }
-
-    public static void printWarnings(PrintWriter pw, List<ExecutionWarning> warnings) {
-        pw.print("\t\"");
-        pw.print(AbstractQueryApiServlet.ResultFields.WARNINGS.str());
-        pw.print("\": [");
-        for (int i = 0; i < warnings.size(); i++) {
-            final ExecutionWarning warning = warnings.get(i);
-            pw.print("{ \n\t");
-            printField(pw, QueryServiceServlet.ErrorField.CODE.str(), warning.getCode());
-            pw.print("\t");
-            printField(pw, QueryServiceServlet.ErrorField.MSG.str(), JSONUtil.escape(warning.getMessage()), false);
-            pw.print("\t} \n\t");
-            boolean lastWarning = i == warnings.size() - 1;
-            if (!lastWarning) {
-                pw.print(",");
-            }
-        }
-        pw.print("],\n");
-    }
-
     public static void printField(PrintWriter pw, String name, String value) {
         printField(pw, name, value, true);
     }
@@ -185,7 +114,6 @@
         if (comma) {
             pw.print(',');
         }
-        pw.print('\n');
     }
 
     public static ObjectNode getErrorResponse(int errorCode, String errorMessage, String errorSummary,
@@ -359,7 +287,7 @@
             @Override
             public AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException {
                 app.append("\t\"");
-                app.append(AbstractQueryApiServlet.ResultFields.RESULTS.str());
+                app.append(ResultsPrinter.FIELD_NAME);
                 if (resultNo >= 0) {
                     app.append('-').append(String.valueOf(resultNo));
                 }
@@ -371,17 +299,17 @@
     }
 
     public static SessionOutput.ResultDecorator createPostResultDecorator() {
-        return app -> app.append("\t,\n");
+        return app -> app.append("\t");
     }
 
     public static SessionOutput.ResultAppender createResultHandleAppender(String handleUrl) {
-        return (app, handle) -> app.append("\t\"").append(AbstractQueryApiServlet.ResultFields.HANDLE.str())
-                .append("\": \"").append(handleUrl).append(handle).append("\",\n");
+        return (app, handle) -> app.append("\t\"").append(ResultHandlePrinter.FIELD_NAME).append("\": \"")
+                .append(handleUrl).append(handle).append("\"");
     }
 
     public static SessionOutput.ResultAppender createResultStatusAppender() {
-        return (app, status) -> app.append("\t\"").append(AbstractQueryApiServlet.ResultFields.STATUS.str())
-                .append("\": \"").append(status).append("\",\n");
+        return (app, status) -> app.append("\t\"").append(StatusPrinter.FIELD_NAME).append("\": \"").append(status)
+                .append("\"");
     }
 
     public static class ParseOnlyResult {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
index a5c8645..1f8e44c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
@@ -25,6 +25,7 @@
 import java.util.UUID;
 
 import org.apache.asterix.api.common.APIFramework;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.api.RequestReference;
 import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -128,7 +129,7 @@
         SessionOutput output = new SessionOutput(conf, writer);
 
         IStatementExecutor translator = statementExecutorFactory.create(appCtx, statements, output, compilationProvider,
-                storageComponentProvider);
+                storageComponentProvider, new ResponsePrinter(output));
         final RequestReference requestReference =
                 RequestReference.of(UUID.randomUUID().toString(), "CC", System.currentTimeMillis());
         final IRequestParameters requestParameters = new RequestParameters(requestReference, statement, null,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index a89728c..b0b94c5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.algebra.base.ILangExtension;
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.api.IRequestReference;
@@ -130,7 +131,7 @@
             IStatementExecutor.ResultMetadata outMetadata = new IStatementExecutor.ResultMetadata();
             MetadataManager.INSTANCE.init();
             IStatementExecutor translator = statementExecutorFactory.create(ccAppCtx, statements, sessionOutput,
-                    compilationProvider, storageComponentProvider);
+                    compilationProvider, storageComponentProvider, new ResponsePrinter(sessionOutput));
             final IStatementExecutor.Stats stats = new IStatementExecutor.Stats();
             Map<String, IAObject> stmtParams = RequestParameters.deserializeParameterValues(statementParameters);
             final IRequestParameters requestParameters =
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java
new file mode 100644
index 0000000..c57e61d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java
@@ -0,0 +1,57 @@
+/*
+ * 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.app.result;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.ICodedMessage;
+import org.apache.asterix.lang.aql.parser.TokenMgrError;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksException;
+
+public class ExecutionError implements ICodedMessage {
+
+    private final int code;
+    private final String message;
+
+    private ExecutionError(int code, String message) {
+        this.code = code;
+        this.message = message;
+    }
+
+    public static ExecutionError of(Throwable t) {
+        Throwable rootCause = ResultUtil.getRootCause(t);
+        String msg = rootCause.getMessage();
+        if (!(rootCause instanceof AlgebricksException || rootCause instanceof HyracksException
+                || rootCause instanceof TokenMgrError
+                || rootCause instanceof org.apache.asterix.aqlplus.parser.TokenMgrError)) {
+            msg = rootCause.getClass().getSimpleName() + (msg == null ? "" : ": " + msg);
+        }
+        return new ExecutionError(1, msg);
+    }
+
+    @Override
+    public int getCode() {
+        return code;
+    }
+
+    @Override
+    public String getMessage() {
+        return message;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
similarity index 86%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
index baaa5bd..29eb098 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
@@ -16,9 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.app.result;
 
-public class ExecutionWarning {
+import org.apache.asterix.common.api.ICodedMessage;
+
+public class ExecutionWarning implements ICodedMessage {
 
     private final int code;
     private final String message;
@@ -28,10 +30,12 @@
         this.message = message;
     }
 
+    @Override
     public int getCode() {
         return code;
     }
 
+    @Override
     public String getMessage() {
         return message;
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java
new file mode 100644
index 0000000..666b759
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java
@@ -0,0 +1,74 @@
+/*
+ * 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.app.result;
+
+public class ResponseMertics {
+
+    private long elapsedTime;
+    private long executionTime;
+    private long resultCount;
+    private long resultSize;
+    private long processedObjects;
+    private long errorCount;
+    private long warnCount;
+
+    private ResponseMertics() {
+    }
+
+    public static ResponseMertics of(long elapsedTime, long executionTime, long resultCount, long resultSize,
+            long processedObjects, long errorCount, long warnCount) {
+        ResponseMertics mertics = new ResponseMertics();
+        mertics.elapsedTime = elapsedTime;
+        mertics.executionTime = executionTime;
+        mertics.resultCount = resultCount;
+        mertics.resultSize = resultSize;
+        mertics.processedObjects = processedObjects;
+        mertics.errorCount = errorCount;
+        mertics.warnCount = warnCount;
+        return mertics;
+    }
+
+    public long getElapsedTime() {
+        return elapsedTime;
+    }
+
+    public long getExecutionTime() {
+        return executionTime;
+    }
+
+    public long getResultCount() {
+        return resultCount;
+    }
+
+    public long getResultSize() {
+        return resultSize;
+    }
+
+    public long getProcessedObjects() {
+        return processedObjects;
+    }
+
+    public long getErrorCount() {
+        return errorCount;
+    }
+
+    public long getWarnCount() {
+        return warnCount;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java
new file mode 100644
index 0000000..bcd8df5
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java
@@ -0,0 +1,107 @@
+/*
+ * 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.app.result;
+
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.common.api.IResponsePrinter;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ResponsePrinter implements IResponsePrinter {
+
+    private final SessionOutput sessionOutput;
+    private final List<IResponseFieldPrinter> headers = new ArrayList<>();
+    private final List<IResponseFieldPrinter> results = new ArrayList<>();
+    private final List<IResponseFieldPrinter> footers = new ArrayList<>();
+    private boolean headersPrinterd = false;
+    private boolean resultPrinted = false;
+
+    public ResponsePrinter(SessionOutput sessionOutput) {
+        this.sessionOutput = sessionOutput;
+    }
+
+    @Override
+    public void begin() {
+        sessionOutput.hold();
+        sessionOutput.out().print("{\n");
+    }
+
+    @Override
+    public void addHeaderPrinter(IResponseFieldPrinter printer) {
+        headers.add(printer);
+    }
+
+    @Override
+    public void addResultPrinter(IResponseFieldPrinter printer) {
+        results.add(printer);
+    }
+
+    @Override
+    public void addFooterPrinter(IResponseFieldPrinter printer) {
+        footers.add(printer);
+    }
+
+    @Override
+    public void printHeaders() throws HyracksDataException {
+        print(headers);
+        headersPrinterd = !headers.isEmpty();
+    }
+
+    @Override
+    public void printResults() throws HyracksDataException {
+        sessionOutput.release();
+        print(results);
+        resultPrinted = !results.isEmpty();
+        results.clear();
+    }
+
+    @Override
+    public void printFooters() throws HyracksDataException {
+        print(footers);
+    }
+
+    @Override
+    public void end() {
+        sessionOutput.out().print("\n}\n");
+        sessionOutput.release();
+        sessionOutput.out().flush();
+    }
+
+    private void print(List<IResponseFieldPrinter> printers) throws HyracksDataException {
+        final int fieldsCount = printers.size();
+        if ((headersPrinterd || resultPrinted) && fieldsCount > 0) {
+            printFieldSeparator(sessionOutput.out());
+        }
+        for (int i = 0; i < printers.size(); i++) {
+            IResponseFieldPrinter printer = printers.get(i);
+            printer.print(sessionOutput.out());
+            if (i + 1 != fieldsCount) {
+                printFieldSeparator(sessionOutput.out());
+            }
+        }
+    }
+
+    public static void printFieldSeparator(PrintWriter pw) {
+        pw.print(",\n");
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
index 6f9fc47..330cf4b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
@@ -62,6 +62,6 @@
 
     @Override
     public String toString() {
-        return Long.toString(jobId.getId()) + "-" + Long.toString(resultSetId.getId());
+        return jobId.getId() + "-" + resultSetId.getId();
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java
new file mode 100644
index 0000000..855155c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java
@@ -0,0 +1,71 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+import java.util.List;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.ICodedMessage;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.hyracks.util.JSONUtil;
+
+public abstract class AbstractCodedMessagePrinter implements IResponseFieldPrinter {
+
+    private enum CodedMessageField {
+        CODE("code"),
+        MSG("msg");
+
+        private final String str;
+
+        CodedMessageField(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    private final List<ICodedMessage> messages;
+
+    public AbstractCodedMessagePrinter(List<ICodedMessage> messages) {
+        this.messages = messages;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.print("\t\"");
+        pw.print(getName());
+        pw.print("\": [");
+        for (int i = 0; i < messages.size(); i++) {
+            final ICodedMessage codedMessage = messages.get(i);
+            pw.print("{ \n\t");
+            ResultUtil.printField(pw, CodedMessageField.CODE.str(), codedMessage.getCode());
+            pw.print("\t");
+            ResultUtil.printField(pw, CodedMessageField.MSG.str(), JSONUtil.escape(codedMessage.getMessage()), false);
+            pw.print("\t} \n\t");
+            boolean lastWarning = i == messages.size() - 1;
+            if (!lastWarning) {
+                pw.print(",");
+            }
+        }
+        pw.print("]");
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java
new file mode 100644
index 0000000..c83210d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class ClientContextIdPrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "clientContextID";
+    private final String clientContextId;
+
+    public ClientContextIdPrinter(String clientContextId) {
+        this.clientContextId = clientContextId;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        ResultUtil.printField(pw, FIELD_NAME, clientContextId, false);
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
similarity index 67%
copy from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
copy to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
index baaa5bd..cecc8e5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
@@ -16,23 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.app.result.fields;
 
-public class ExecutionWarning {
+import java.util.List;
 
-    private final int code;
-    private final String message;
+import org.apache.asterix.common.api.ICodedMessage;
 
-    public ExecutionWarning(int code, String message) {
-        this.code = code;
-        this.message = message;
+public class ErrorsPrinter extends AbstractCodedMessagePrinter {
+
+    private static final String FIELD_NAME = "errors";
+
+    public ErrorsPrinter(List<ICodedMessage> errors) {
+        super(errors);
     }
 
-    public int getCode() {
-        return code;
-    }
-
-    public String getMessage() {
-        return message;
+    @Override
+    public String getName() {
+        return FIELD_NAME;
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
new file mode 100644
index 0000000..35b5f43
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
@@ -0,0 +1,100 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.ResponseMertics;
+import org.apache.asterix.common.api.Duration;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class MetricsPrinter implements IResponseFieldPrinter {
+
+    public enum Metrics {
+        ELAPSED_TIME("elapsedTime"),
+        EXECUTION_TIME("executionTime"),
+        RESULT_COUNT("resultCount"),
+        RESULT_SIZE("resultSize"),
+        ERROR_COUNT("errorCount"),
+        PROCESSED_OBJECTS_COUNT("processedObjects"),
+        WARNING_COUNT("warningCount");
+
+        private final String str;
+
+        Metrics(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    public static final String FIELD_NAME = "metrics";
+    private final ResponseMertics mertics;
+    private final Charset resultCharset;
+
+    public MetricsPrinter(ResponseMertics mertics, Charset resultCharset) {
+        this.mertics = mertics;
+        this.resultCharset = resultCharset;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        boolean useAscii = !StandardCharsets.UTF_8.equals(resultCharset)
+                && !"μ".contentEquals(resultCharset.decode(resultCharset.encode("μ")));
+        pw.print("\t\"");
+        pw.print(FIELD_NAME);
+        pw.print("\": {\n");
+        pw.print("\t");
+        ResultUtil.printField(pw, Metrics.ELAPSED_TIME.str(), Duration.formatNanos(mertics.getElapsedTime(), useAscii));
+        pw.print("\n\t");
+        ResultUtil.printField(pw, Metrics.EXECUTION_TIME.str(),
+                Duration.formatNanos(mertics.getExecutionTime(), useAscii));
+        pw.print("\n\t");
+        ResultUtil.printField(pw, Metrics.RESULT_COUNT.str(), mertics.getResultCount(), true);
+        pw.print("\n\t");
+        ResultUtil.printField(pw, Metrics.RESULT_SIZE.str(), mertics.getResultSize(), true);
+        pw.print("\n\t");
+        final boolean hasErrors = mertics.getErrorCount() > 0;
+        final boolean hasWarnings = mertics.getWarnCount() > 0;
+        ResultUtil.printField(pw, Metrics.PROCESSED_OBJECTS_COUNT.str(), mertics.getProcessedObjects(),
+                hasWarnings || hasErrors);
+        pw.print("\n");
+        if (hasWarnings) {
+            pw.print("\t");
+            ResultUtil.printField(pw, Metrics.WARNING_COUNT.str(), mertics.getWarnCount(), hasErrors);
+            pw.print("\n");
+        }
+        if (hasErrors) {
+            pw.print("\t");
+            ResultUtil.printField(pw, Metrics.ERROR_COUNT.str(), mertics.getErrorCount(), false);
+            pw.print("\n");
+        }
+        pw.print("\t}");
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
new file mode 100644
index 0000000..2989d2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+import java.util.List;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.message.ExecuteStatementResponseMessage;
+import org.apache.asterix.app.result.ResponsePrinter;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.api.result.ResultSetId;
+
+public class NcResultPrinter implements IResponseFieldPrinter {
+
+    private final IStatementExecutor.ResultDelivery delivery;
+    private final ExecuteStatementResponseMessage responseMsg;
+    private final IApplicationContext appCtx;
+    private final IResultSet resultSet;
+    private final SessionOutput sessionOutput;
+
+    public NcResultPrinter(IApplicationContext appCtx, ExecuteStatementResponseMessage responseMsg,
+            IResultSet resultSet, IStatementExecutor.ResultDelivery delivery, SessionOutput sessionOutput) {
+        this.appCtx = appCtx;
+        this.responseMsg = responseMsg;
+        this.delivery = delivery;
+        this.resultSet = resultSet;
+        this.sessionOutput = sessionOutput;
+    }
+
+    @Override
+    public void print(PrintWriter pw) throws HyracksDataException {
+        IStatementExecutor.ResultMetadata resultMetadata = responseMsg.getMetadata();
+        List<Triple<JobId, ResultSetId, ARecordType>> resultSets = resultMetadata.getResultSets();
+        if (delivery == IStatementExecutor.ResultDelivery.IMMEDIATE && !resultSets.isEmpty()) {
+            final IStatementExecutor.Stats stats = responseMsg.getStats();
+            stats.setProcessedObjects(responseMsg.getStats().getProcessedObjects());
+            for (int i = 0; i < resultSets.size(); i++) {
+                Triple<JobId, ResultSetId, ARecordType> rsmd = resultSets.get(i);
+                ResultReader resultReader = new ResultReader(resultSet, rsmd.getLeft(), rsmd.getMiddle());
+                ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, rsmd.getRight());
+                if (i + 1 != resultSets.size()) {
+                    ResponsePrinter.printFieldSeparator(pw);
+                }
+            }
+        } else {
+            pw.append(responseMsg.getResult());
+        }
+    }
+
+    @Override
+    public String getName() {
+        return ResultsPrinter.FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java
new file mode 100644
index 0000000..d7ee269
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java
@@ -0,0 +1,46 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class ParseOnlyResultPrinter implements IResponseFieldPrinter {
+
+    private final ResultUtil.ParseOnlyResult parseOnlyResult;
+
+    public ParseOnlyResultPrinter(ResultUtil.ParseOnlyResult parseOnlyResult) {
+        this.parseOnlyResult = parseOnlyResult;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.print("\t\"");
+        pw.print(getName());
+        pw.print("\":");
+        pw.print(parseOnlyResult.asJson());
+    }
+
+    @Override
+    public String getName() {
+        return ResultsPrinter.FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java
new file mode 100644
index 0000000..4021956
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java
@@ -0,0 +1,58 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ExecutionPlansJsonPrintUtil;
+import org.apache.asterix.translator.SessionConfig;
+
+public class PlansPrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "plans";
+    private final ExecutionPlans executionPlans;
+    private final SessionConfig.PlanFormat planFormat;
+
+    public PlansPrinter(ExecutionPlans executionPlans, SessionConfig.PlanFormat planFormat) {
+        this.executionPlans = executionPlans;
+        this.planFormat = planFormat;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.print("\t\"");
+        pw.print(FIELD_NAME);
+        pw.print("\":");
+        switch (planFormat) {
+            case JSON:
+            case STRING:
+                pw.print(ExecutionPlansJsonPrintUtil.asJson(executionPlans, planFormat));
+                break;
+            default:
+                throw new IllegalStateException("Unrecognized plan format: " + planFormat);
+        }
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java
new file mode 100644
index 0000000..b66cbfb
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java
@@ -0,0 +1,45 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class RequestIdPrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "requestID";
+
+    private final String requestId;
+
+    public RequestIdPrinter(String requestId) {
+        this.requestId = requestId;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        ResultUtil.printField(pw, FIELD_NAME, requestId, false);
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java
new file mode 100644
index 0000000..830c751
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java
@@ -0,0 +1,65 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.ResultHandle;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ResultHandlePrinter implements IResponseFieldPrinter {
+
+    public static final String FIELD_NAME = "handle";
+    private final SessionOutput sessionOutput;
+    private final String handle;
+
+    public ResultHandlePrinter(SessionOutput sessionOutput, ResultHandle handle) {
+        this.sessionOutput = sessionOutput;
+        this.handle = handle.toString();
+    }
+
+    public ResultHandlePrinter(String handle) {
+        this.handle = handle;
+        sessionOutput = null;
+    }
+
+    @Override
+    public void print(PrintWriter pw) throws HyracksDataException {
+        if (sessionOutput != null) {
+            final AlgebricksAppendable app = new AlgebricksAppendable(pw);
+            try {
+                sessionOutput.appendHandle(app, handle);
+            } catch (AlgebricksException e) {
+                throw HyracksDataException.create(e);
+            }
+        } else {
+            ResultUtil.printField(pw, FIELD_NAME, handle, false);
+        }
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
new file mode 100644
index 0000000..52198de
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ResultsPrinter implements IResponseFieldPrinter {
+
+    public static final String FIELD_NAME = "results";
+    private final IApplicationContext appCtx;
+    private final ARecordType recordType;
+    private final ResultReader resultReader;
+    private final IStatementExecutor.Stats stats;
+    private final SessionOutput sessionOutput;
+
+    public ResultsPrinter(IApplicationContext appCtx, ResultReader resultReader, ARecordType recordType,
+            IStatementExecutor.Stats stats, SessionOutput sessionOutput) {
+        this.appCtx = appCtx;
+        this.recordType = recordType;
+        this.resultReader = resultReader;
+        this.stats = stats;
+        this.sessionOutput = sessionOutput;
+    }
+
+    @Override
+    public void print(PrintWriter pw) throws HyracksDataException {
+        ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, recordType);
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
new file mode 100644
index 0000000..25e3742
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
@@ -0,0 +1,47 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class SignaturePrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "signature";
+
+    public SignaturePrinter() {
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.print("\t\"");
+        pw.print(FIELD_NAME);
+        pw.print("\": {\n");
+        pw.print("\t");
+        ResultUtil.printField(pw, "*", "*", false);
+        pw.print("\n\t}");
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java
new file mode 100644
index 0000000..372ea45
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.AbstractQueryApiServlet;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class StatusPrinter implements IResponseFieldPrinter {
+
+    public static final String FIELD_NAME = "status";
+    private final AbstractQueryApiServlet.ResultStatus status;
+
+    public StatusPrinter(AbstractQueryApiServlet.ResultStatus status) {
+        this.status = status;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.append("\t\"").append(FIELD_NAME).append("\": \"").append(status.str()).append("\"");
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java
new file mode 100644
index 0000000..788fbcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.app.result.fields;
+
+import static org.apache.hyracks.http.server.utils.HttpUtil.ContentType.CSV;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.hyracks.http.server.utils.HttpUtil;
+
+public class TypePrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "type";
+    private final SessionConfig sessionConfig;
+
+    public TypePrinter(SessionConfig sessionConfig) {
+        this.sessionConfig = sessionConfig;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        switch (sessionConfig.fmt()) {
+            case ADM:
+                ResultUtil.printField(pw, FIELD_NAME, HttpUtil.ContentType.APPLICATION_ADM, false);
+                break;
+            case CSV:
+                String contentType =
+                        CSV + "; header=" + (sessionConfig.is(SessionConfig.FORMAT_CSV_HEADER) ? "present" : "absent");
+                ResultUtil.printField(pw, FIELD_NAME, contentType, false);
+                break;
+            default:
+                break;
+        }
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
similarity index 66%
copy from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
copy to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
index baaa5bd..2ebe5f4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
@@ -16,23 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.app.result.fields;
 
-public class ExecutionWarning {
+import java.util.List;
 
-    private final int code;
-    private final String message;
+import org.apache.asterix.common.api.ICodedMessage;
 
-    public ExecutionWarning(int code, String message) {
-        this.code = code;
-        this.message = message;
+public class WarningsPrinter extends AbstractCodedMessagePrinter {
+
+    private static final String FIELD_NAME = "warnings";
+
+    public WarningsPrinter(List<ICodedMessage> warnings) {
+        super(warnings);
     }
 
-    public int getCode() {
-        return code;
-    }
-
-    public String getMessage() {
-        return message;
+    @Override
+    public String getName() {
+        return FIELD_NAME;
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
index f6bd708..3e47362 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
@@ -22,6 +22,7 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -49,7 +50,8 @@
 
     @Override
     public IStatementExecutor create(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
-            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider) {
-        return new QueryTranslator(appCtx, statements, output, compilationProvider, executorService);
+            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider,
+            IResponsePrinter responsePrinter) {
+        return new QueryTranslator(appCtx, statements, output, compilationProvider, executorService, responsePrinter);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 8c0cc98..ab965ad 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -45,15 +45,20 @@
 import org.apache.asterix.api.common.APIFramework;
 import org.apache.asterix.api.http.server.AbstractQueryApiServlet;
 import org.apache.asterix.api.http.server.ApiServlet;
-import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.app.active.ActiveEntityEventsListener;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.active.FeedEventsListener;
+import org.apache.asterix.app.result.ExecutionError;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.fields.ErrorsPrinter;
+import org.apache.asterix.app.result.fields.ResultHandlePrinter;
+import org.apache.asterix.app.result.fields.ResultsPrinter;
+import org.apache.asterix.app.result.fields.StatusPrinter;
 import org.apache.asterix.common.api.IClientRequest;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.api.IRequestTracker;
+import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
@@ -224,9 +229,11 @@
     protected final ExecutorService executorService;
     protected final EnumSet<JobFlag> jobFlags = EnumSet.noneOf(JobFlag.class);
     protected final IMetadataLockManager lockManager;
+    protected final IResponsePrinter responsePrinter;
 
     public QueryTranslator(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
-            ILangCompilationProvider compilationProvider, ExecutorService executorService) {
+            ILangCompilationProvider compilationProvider, ExecutorService executorService,
+            IResponsePrinter responsePrinter) {
         this.appCtx = appCtx;
         this.lockManager = appCtx.getMetadataLockManager();
         this.statements = statements;
@@ -238,6 +245,7 @@
         rewriterFactory = compilationProvider.getRewriterFactory();
         activeDataverse = MetadataBuiltinEntities.DEFAULT_DATAVERSE;
         this.executorService = executorService;
+        this.responsePrinter = responsePrinter;
         if (appCtx.getServiceContext().getAppConfig().getBoolean(CCConfig.Option.ENFORCE_FRAME_WRITER_PROTOCOL)) {
             this.jobFlags.add(JobFlag.ENFORCE_CONTRACT);
         }
@@ -2506,16 +2514,17 @@
                 createAndRunJob(hcc, jobFlags, null, compiler, locker, resultDelivery, id -> {
                     final ResultReader resultReader = new ResultReader(resultSet, id, resultSetId);
                     updateJobStats(id, stats, metadataProvider.getResultSetId());
-                    // stop buffering and allow for streaming result delivery
-                    sessionOutput.release();
-                    ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats,
-                            metadataProvider.findOutputRecordType());
+                    responsePrinter.addResultPrinter(new ResultsPrinter(appCtx, resultReader,
+                            metadataProvider.findOutputRecordType(), stats, sessionOutput));
+                    responsePrinter.printResults();
                 }, requestParameters, cancellable, appCtx, metadataProvider);
                 break;
             case DEFERRED:
                 createAndRunJob(hcc, jobFlags, null, compiler, locker, resultDelivery, id -> {
                     updateJobStats(id, stats, metadataProvider.getResultSetId());
-                    ResultUtil.printResultHandle(sessionOutput, new ResultHandle(id, resultSetId));
+                    responsePrinter.addResultPrinter(
+                            new ResultHandlePrinter(sessionOutput, new ResultHandle(id, resultSetId)));
+                    responsePrinter.printResults();
                     if (outMetadata != null) {
                         outMetadata.getResultSets()
                                 .add(Triple.of(id, resultSetId, metadataProvider.findOutputRecordType()));
@@ -2543,8 +2552,9 @@
         try {
             createAndRunJob(hcc, jobFlags, jobId, compiler, locker, resultDelivery, id -> {
                 final ResultHandle handle = new ResultHandle(id, resultSetId);
-                ResultUtil.printStatus(sessionOutput, AbstractQueryApiServlet.ResultStatus.RUNNING);
-                ResultUtil.printResultHandle(sessionOutput, handle);
+                responsePrinter.addResultPrinter(new StatusPrinter(AbstractQueryApiServlet.ResultStatus.RUNNING));
+                responsePrinter.addResultPrinter(new ResultHandlePrinter(sessionOutput, handle));
+                responsePrinter.printResults();
                 synchronized (printed) {
                     printed.setTrue();
                     printed.notify();
@@ -2553,8 +2563,13 @@
         } catch (Exception e) {
             if (Objects.equals(JobId.INVALID, jobId.getValue())) {
                 // compilation failed
-                ResultUtil.printStatus(sessionOutput, AbstractQueryApiServlet.ResultStatus.FAILED);
-                ResultUtil.printError(sessionOutput.out(), e);
+                responsePrinter.addResultPrinter(new StatusPrinter(AbstractQueryApiServlet.ResultStatus.FAILED));
+                responsePrinter.addResultPrinter(new ErrorsPrinter(Collections.singletonList(ExecutionError.of(e))));
+                try {
+                    responsePrinter.printResults();
+                } catch (HyracksDataException ex) {
+                    LOGGER.error("failed to print result", ex);
+                }
             } else {
                 GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR,
                         resultDelivery.name() + " job with id " + jobId.getValue() + " " + "failed", e);
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
index debcd19..b74f4c6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -29,6 +29,7 @@
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.app.translator.QueryTranslator;
 import org.apache.asterix.common.cluster.IClusterStateManager;
@@ -438,7 +439,7 @@
         List<Statement> stmts = new ArrayList<>();
         DefaultStatementExecutorFactory qtFactory = new DefaultStatementExecutorFactory();
         IStatementExecutor translator = qtFactory.create(metadataProvider.getApplicationContext(), stmts, sessionOutput,
-                new SqlppCompilationProvider(), new StorageComponentProvider());
+                new SqlppCompilationProvider(), new StorageComponentProvider(), new ResponsePrinter(sessionOutput));
         return translator;
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
index 02bbcf5..fe64dd1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
@@ -23,9 +23,11 @@
 import java.io.ByteArrayOutputStream;
 import java.io.PrintWriter;
 import java.nio.charset.StandardCharsets;
+import java.util.Collections;
 
-import org.apache.asterix.api.http.server.AbstractQueryApiServlet;
 import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.fields.ErrorsPrinter;
+import org.apache.asterix.app.result.fields.MetricsPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.test.common.ResultExtractor;
@@ -65,7 +67,8 @@
         try {
             rs.print(resultReader);
         } catch (RuntimeException e) {
-            ResultUtil.printError(out, e, true);
+            final ExecutionError error = ExecutionError.of(e);
+            new ErrorsPrinter(Collections.singletonList(error)).print(out);
             printMetrics(out, 1);
         }
         out.print("}");
@@ -98,7 +101,7 @@
 
     private static void printMetrics(PrintWriter pw, long errorCount) {
         pw.print("\t\"");
-        pw.print(AbstractQueryApiServlet.ResultFields.METRICS.str());
+        pw.print(MetricsPrinter.FIELD_NAME);
         pw.print("\": {\n");
         ResultUtil.printField(pw, "errorCount", errorCount, false);
         pw.print("\t}\n");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index dfb92b7..80dde8a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -36,6 +36,7 @@
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
@@ -94,10 +95,12 @@
 
         // Mock MetadataProvider
         CCExtensionManager extensionManager = (CCExtensionManager) appCtx.getExtensionManager();
+        SessionOutput sessionOutput = Mockito.mock(SessionOutput.class);
         IStatementExecutor statementExecutor = extensionManager
                 .getStatementExecutorFactory(appCtx.getServiceContext().getControllerService().getExecutor())
-                .create(appCtx, Collections.emptyList(), Mockito.mock(SessionOutput.class),
-                        extensionManager.getCompilationProvider(Language.SQLPP), appCtx.getStorageComponentProvider());
+                .create(appCtx, Collections.emptyList(), sessionOutput,
+                        extensionManager.getCompilationProvider(Language.SQLPP), appCtx.getStorageComponentProvider(),
+                        new ResponsePrinter(sessionOutput));
         MetadataProvider mdProvider = new MetadataProvider(appCtx, null);
         // Add event listener
         ActiveEntityEventsListener eventsListener = new DummyFeedEventsListener(statementExecutor, appCtx, null,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
similarity index 67%
copy from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
index baaa5bd..06202c4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
@@ -16,23 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.common.api;
 
-public class ExecutionWarning {
+public interface ICodedMessage {
 
-    private final int code;
-    private final String message;
+    /**
+     * Gets the code of the message
+     *
+     * @return the code
+     */
+    int getCode();
 
-    public ExecutionWarning(int code, String message) {
-        this.code = code;
-        this.message = message;
-    }
-
-    public int getCode() {
-        return code;
-    }
-
-    public String getMessage() {
-        return message;
-    }
+    /**
+     * Gets the message
+     *
+     * @return the message
+     */
+    String getMessage();
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
similarity index 63%
copy from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
index baaa5bd..d2006cb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
@@ -16,23 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.common.api;
 
-public class ExecutionWarning {
+import java.io.PrintWriter;
 
-    private final int code;
-    private final String message;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-    public ExecutionWarning(int code, String message) {
-        this.code = code;
-        this.message = message;
-    }
+public interface IResponseFieldPrinter {
 
-    public int getCode() {
-        return code;
-    }
+    /**
+     * Prints this field using {@code pw}
+     *
+     * @param pw
+     * @throws HyracksDataException
+     */
+    void print(PrintWriter pw) throws HyracksDataException;
 
-    public String getMessage() {
-        return message;
-    }
+    /**
+     * Gets the name of this field
+     *
+     * @return the name of the field
+     */
+    String getName();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java
new file mode 100644
index 0000000..a4dc7b5
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java
@@ -0,0 +1,76 @@
+/*
+ * 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.common.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IResponsePrinter {
+
+    /**
+     * Performs any operations required before printing the response fields.
+     */
+    void begin();
+
+    /**
+     * Adds a response header printer.
+     *
+     * @param printer
+     */
+    void addHeaderPrinter(IResponseFieldPrinter printer);
+
+    /**
+     * Adds a response result printer.
+     *
+     * @param printer
+     */
+    void addResultPrinter(IResponseFieldPrinter printer);
+
+    /**
+     * Adds a response footer printer.
+     *
+     * @param printer
+     */
+    void addFooterPrinter(IResponseFieldPrinter printer);
+
+    /**
+     * Prints the added response headers.
+     *
+     * @throws HyracksDataException
+     */
+    void printHeaders() throws HyracksDataException;
+
+    /**
+     * Prints the added response results.
+     *
+     * @throws HyracksDataException
+     */
+    void printResults() throws HyracksDataException;
+
+    /**
+     * Prints the added response footers.
+     *
+     * @throws HyracksDataException
+     */
+    void printFooters() throws HyracksDataException;
+
+    /**
+     * Performs any operations required after printing the response fields.
+     */
+    void end();
+}

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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newchange
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/E7Ur1Ckz55vUzL1dA : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:43:34 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/11318/ (15/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:27 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5742/ (1/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:52 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/6050/ (12/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:52 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8604/


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:37:11 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/6200/ (5/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:41 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5825/ (10/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:35:00 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4453/


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:39:02 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/KaaSoQdShpiBKD1m9 : UNSTABLE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 11:58:25 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/11319/ (8/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:45 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Murtadha Hubail (Code Review)" <de...@asterixdb.apache.org>.
Hello Anon. E. Moose (1000171), Jenkins, 

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

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

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

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................

[NO ISSUE][OTH] Introduce ResponsePrinter

- user model changes: no
- storage format changes: no
- interface changes: yes

Details:
- Introduce ResponsePrinter that is responsible for printing
  the returned response from query service.
- Make ResponsePrinter responsible for adding separators between
  fields in the returned response.
- Introduce IResponseFieldPrinter and encapsulate each response
  field in its own printer.
- Introduce ICodedMessage and use it for any object that
  has code<->message (e.g errors and warnings).

Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java
C asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ExplainOnlyResultsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java
C asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
C asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
C asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java
41 files changed, 1,354 insertions(+), 421 deletions(-)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/yjsm3WBaxaM8Lo34A : UNSTABLE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 05:40:14 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/6049/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:08 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9785/ (5/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:10 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/6199/ (9/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:16 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5993/ (9/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:58 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8606/


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:37:37 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/1006/ (4/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:40 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Murtadha Hubail (Code Review)" <de...@asterixdb.apache.org>.
Hello Anon. E. Moose (1000171), Jenkins, 

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

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

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

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................

[NO ISSUE][OTH] Introduce ResponsePrinter

- user model changes: no
- storage format changes: no
- interface changes: yes

Details:
- Introduce ResponsePrinter that is responsible for printing
  the returned response from query service.
- Make ResponsePrinter responsible for adding separators between
  fields in the returned response.
- Introduce IResponseFieldPrinter and encapsulate each response
  field in its own printer.
- Introduce ICodedMessage and use it for any object that
  has code<->message (e.g errors and warnings).

Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java
C asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ExplainOnlyResultsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java
C asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
C asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
C asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java
41 files changed, 1,356 insertions(+), 420 deletions(-)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9784/ (15/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:35:09 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5806/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:35:08 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1194/ (6/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:11 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5844/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:26 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4452/


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:59:36 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5994/ (10/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:18 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5744/ (1/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:38 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/1102/ (4/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:53 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/1104/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:54 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1: Integration-Tests-1

Integration Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8604/ : UNSTABLE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 01:50:19 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3838/ (2/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:08 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5843/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:35:05 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Integration Tests Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8605/


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:59:41 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/766/ (8/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:57 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/5995/ (6/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:42 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/6048/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:35:11 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/GUtBUUKXjWT7b28C8 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:39:00 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3839/ (2/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:39 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/767/ (11/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:19 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5826/ (12/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:21 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6409/ (7/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:12 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/5743/ (1/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:08 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/609/ (10/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:48 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6408/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:53 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Tests Failed
https://cbjenkins.page.link/3EgJ6urkvkvaNVWR6 : UNSTABLE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:20:45 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Till Westmann (Code Review)" <de...@asterixdb.apache.org>.
Till Westmann has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3: Code-Review+2

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/3410/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
File asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java:

https://asterix-gerrit.ics.uci.edu/#/c/3410/3/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java@65
PS3, Line 65: \t
In a future version we should either parameterize the ResponseFieldPrinters to allow for indeed or non-indented output or had different printers for both cases.



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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Wed, 29 May 2019 05:40:07 +0000
Gerrit-HasComments: Yes

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/6410/ (14/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:55 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5807/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:29 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/5808/ (3/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:39 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/607/ (12/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:35:03 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1: Contrib-2

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4451/ : FAILURE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:47:42 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/6198/ (7/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:55 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Murtadha Hubail (Code Review)" <de...@asterixdb.apache.org>.
Murtadha Hubail has submitted this change and it was merged. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................

[NO ISSUE][OTH] Introduce ResponsePrinter

- user model changes: no
- storage format changes: no
- interface changes: yes

Details:
- Introduce ResponsePrinter that is responsible for printing
  the returned response from query service.
- Make ResponsePrinter responsible for adding separators between
  fields in the returned response.
- Introduce IResponseFieldPrinter and encapsulate each response
  field in its own printer.
- Introduce ICodedMessage and use it for any object that
  has code<->message (e.g errors and warnings).

Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/3410
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java
R asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java
C asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ExplainOnlyResultsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java
A asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java
C asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
M asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
M asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
C asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
C asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
A asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java
41 files changed, 1,356 insertions(+), 420 deletions(-)

Approvals:
  Jenkins: Verified; ; Verified
  Anon. E. Moose (1000171): 
  Till Westmann: Looks good to me, approved

Objections:
  Jenkins: Violations found



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index e00c15a..c27a30f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -24,6 +24,7 @@
 import java.util.List;
 import java.util.Map;
 
+import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.lang.common.base.IStatementRewriter;
@@ -155,4 +156,11 @@
      * @return the executions plans
      */
     ExecutionPlans getExecutionPlans();
+
+    /**
+     * Gets the response printer
+     *
+     * @return the responer printer
+     */
+    IResponsePrinter getResponsePrinter();
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
index b244c0c..26ebbd5 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutorFactory.java
@@ -20,6 +20,7 @@
 
 import java.util.List;
 
+import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -43,8 +44,10 @@
      *            provides query language related components
      * @param storageComponentProvider
      *            provides storage related components
+     * @param responsePrinter
      * @return an implementation of {@code IStatementExecutor} thaxt is used to execute the passed list of statements
      */
     IStatementExecutor create(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
-            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider);
+            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider,
+            IResponsePrinter responsePrinter);
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
index 97125f0..ed59cf4 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/SessionOutput.java
@@ -38,6 +38,10 @@
     private SessionOutput.ResultAppender handleAppender;
     private final SessionOutput.ResultAppender statusAppender;
 
+    public SessionOutput(PrintWriter out) {
+        this(null, out);
+    }
+
     public SessionOutput(SessionConfig config, PrintWriter out) {
         this(config, out, null, null, null, null);
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index d558e0d..d971c1d 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -33,8 +33,9 @@
 
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
-import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.fields.ExplainOnlyResultsPrinter;
 import org.apache.asterix.common.api.INodeJobTracker;
+import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
@@ -74,7 +75,6 @@
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.ExecutionPlans;
-import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
 import org.apache.asterix.translator.SessionOutput;
 import org.apache.asterix.translator.SqlppExpressionToPlanTranslator;
@@ -192,7 +192,7 @@
 
     public JobSpecification compileQuery(IClusterInfoCollector clusterInfoCollector, MetadataProvider metadataProvider,
             Query query, int varCounter, String outputDatasetName, SessionOutput output,
-            ICompiledDmlStatement statement, Map<VarIdentifier, IAObject> externalVars)
+            ICompiledDmlStatement statement, Map<VarIdentifier, IAObject> externalVars, IResponsePrinter printer)
             throws AlgebricksException, ACIDException {
 
         // establish facts
@@ -261,7 +261,7 @@
             }
         }
         if (isExplainOnly) {
-            printPlanAsResult(metadataProvider, output);
+            printPlanAsResult(metadataProvider, output, printer);
             if (!conf.is(SessionConfig.OOB_OPTIMIZED_LOGICAL_PLAN)) {
                 executionPlans.setOptimizedLogicalPlan(null);
             }
@@ -307,13 +307,12 @@
         return spec;
     }
 
-    private void printPlanAsResult(MetadataProvider metadataProvider, SessionOutput output) throws AlgebricksException {
-        final SessionConfig conf = output.config();
-        boolean quoteResult = output.config().getPlanFormat() == SessionConfig.PlanFormat.STRING;
-        conf.set(SessionConfig.FORMAT_QUOTE_RECORD, quoteResult);
+    private void printPlanAsResult(MetadataProvider metadataProvider, SessionOutput output, IResponsePrinter printer)
+            throws AlgebricksException {
         try {
-            ResultUtil.printResults(metadataProvider.getApplicationContext(), executionPlans.getOptimizedLogicalPlan(),
-                    output, new Stats(), null);
+            printer.addResultPrinter(new ExplainOnlyResultsPrinter(metadataProvider.getApplicationContext(),
+                    executionPlans.getOptimizedLogicalPlan(), output));
+            printer.printResults();
         } catch (HyracksDataException e) {
             throw new AlgebricksException(e);
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
index 6f27eb5..4dadf55 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
@@ -20,7 +20,6 @@
 
 import static org.apache.asterix.api.http.server.ServletConstants.HYRACKS_CONNECTION_ATTR;
 
-import java.io.PrintWriter;
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.common.api.IApplicationContext;
@@ -38,30 +37,6 @@
     private static final Logger LOGGER = LogManager.getLogger();
     protected final IApplicationContext appCtx;
 
-    public enum ResultFields {
-        REQUEST_ID("requestID"),
-        CLIENT_ID("clientContextID"),
-        SIGNATURE("signature"),
-        TYPE("type"),
-        STATUS("status"),
-        RESULTS("results"),
-        HANDLE("handle"),
-        ERRORS("errors"),
-        METRICS("metrics"),
-        PLANS("plans"),
-        WARNINGS("warnings");
-
-        private final String str;
-
-        ResultFields(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
     public enum ResultStatus {
         RUNNING("running"),
         SUCCESS("success"),
@@ -72,22 +47,6 @@
         private final String str;
 
         ResultStatus(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
-    public enum ErrorField {
-        CODE("code"),
-        MSG("msg"),
-        STACK("stack");
-
-        private final String str;
-
-        ErrorField(String str) {
             this.str = str;
         }
 
@@ -118,13 +77,4 @@
         }
         return hcc;
     }
-
-    protected static void printRequestId(PrintWriter pw, String requestId) {
-        ResultUtil.printField(pw, ResultFields.REQUEST_ID.str(), requestId);
-    }
-
-    protected static void printHandle(PrintWriter pw, String handle, boolean comma) {
-        ResultUtil.printField(pw, ResultFields.HANDLE.str(), handle, comma);
-    }
-
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
index ea6e616..f395931 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ApiServlet.java
@@ -32,6 +32,7 @@
 
 import javax.imageio.ImageIO;
 
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.api.IRequestReference;
 import org.apache.asterix.common.config.GlobalConfig;
@@ -147,7 +148,7 @@
             SessionOutput sessionOutput = new SessionOutput(sessionConfig, out);
             MetadataManager.INSTANCE.init();
             IStatementExecutor translator = statementExectorFactory.create(appCtx, statements, sessionOutput,
-                    compilationProvider, componentProvider);
+                    compilationProvider, componentProvider, new ResponsePrinter(sessionOutput));
             double duration;
             long startTime = System.currentTimeMillis();
             final IRequestParameters requestParameters = new RequestParameters(requestReference, query, resultSet,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
index 8923ae8..7d7bad2 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/NCQueryServiceServlet.java
@@ -29,7 +29,8 @@
 import org.apache.asterix.app.message.CancelQueryRequest;
 import org.apache.asterix.app.message.ExecuteStatementRequestMessage;
 import org.apache.asterix.app.message.ExecuteStatementResponseMessage;
-import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResponsePrinter;
+import org.apache.asterix.app.result.fields.NcResultPrinter;
 import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.IRequestReference;
@@ -40,14 +41,10 @@
 import org.apache.asterix.common.messaging.api.INCMessageBroker;
 import org.apache.asterix.common.messaging.api.MessageFuture;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
-import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.ResultProperties;
 import org.apache.asterix.translator.SessionOutput;
-import org.apache.commons.lang3.tuple.Triple;
 import org.apache.hyracks.api.application.INCServiceContext;
-import org.apache.hyracks.api.job.JobId;
-import org.apache.hyracks.api.result.ResultSetId;
 import org.apache.hyracks.http.api.IChannelClosedHandler;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.server.HttpServer;
@@ -73,7 +70,8 @@
     protected void executeStatement(IRequestReference requestReference, String statementsText,
             SessionOutput sessionOutput, ResultProperties resultProperties, IStatementExecutor.Stats stats,
             QueryServiceRequestParameters param, RequestExecutionState execution,
-            Map<String, String> optionalParameters, Map<String, byte[]> statementParameters) throws Exception {
+            Map<String, String> optionalParameters, Map<String, byte[]> statementParameters,
+            ResponsePrinter responsePrinter) throws Exception {
         // Running on NC -> send 'execute' message to CC
         INCServiceContext ncCtx = (INCServiceContext) serviceCtx;
         INCMessageBroker ncMb = (INCMessageBroker) ncCtx.getMessageBroker();
@@ -119,20 +117,11 @@
                 throw new Exception(err.toString(), err);
             }
         }
-        // no errors - stop buffering and allow for streaming result delivery
-        sessionOutput.release();
-
-        IStatementExecutor.ResultMetadata resultMetadata = responseMsg.getMetadata();
-        if (delivery == IStatementExecutor.ResultDelivery.IMMEDIATE && !resultMetadata.getResultSets().isEmpty()) {
-            stats.setProcessedObjects(responseMsg.getStats().getProcessedObjects());
-            for (Triple<JobId, ResultSetId, ARecordType> rsmd : resultMetadata.getResultSets()) {
-                ResultReader resultReader = new ResultReader(getResultSet(), rsmd.getLeft(), rsmd.getMiddle());
-                ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, rsmd.getRight());
-            }
-        } else {
-            sessionOutput.out().append(responseMsg.getResult());
+        if (hasResult(responseMsg)) {
+            responsePrinter.addResultPrinter(
+                    new NcResultPrinter(appCtx, responseMsg, getResultSet(), delivery, sessionOutput));
         }
-        printExecutionPlans(sessionOutput, responseMsg.getExecutionPlans());
+        buildResponseResults(responsePrinter, sessionOutput, responseMsg.getExecutionPlans());
     }
 
     private void cancelQuery(INCMessageBroker messageBroker, String nodeId, String uuid, String clientContextID,
@@ -174,4 +163,8 @@
     public IChannelClosedHandler getChannelClosedHandler(HttpServer server) {
         return InterruptOnCloseHandler.INSTANCE;
     }
+
+    private static boolean hasResult(ExecuteStatementResponseMessage responseMsg) {
+        return !responseMsg.getMetadata().getResultSets().isEmpty() || !responseMsg.getResult().isEmpty();
+    }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
index cda4d34..2b06dbb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
@@ -22,8 +22,12 @@
 import java.util.concurrent.ConcurrentMap;
 
 import org.apache.asterix.api.common.ResultMetadata;
+import org.apache.asterix.app.result.ResponseMertics;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.fields.MetricsPrinter;
+import org.apache.asterix.app.result.fields.ResultsPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
@@ -87,15 +91,19 @@
             }
             ResultMetadata metadata = (ResultMetadata) resultReader.getMetadata();
             SessionOutput sessionOutput = initResponse(request, response, metadata.getFormat());
+            ResponsePrinter printer = new ResponsePrinter(sessionOutput);
             if (metadata.getFormat() == SessionConfig.OutputFormat.CLEAN_JSON
                     || metadata.getFormat() == SessionConfig.OutputFormat.LOSSLESS_JSON) {
                 final Stats stats = new Stats();
-                sessionOutput.out().print("{\n");
-                ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, null);
-                QueryServiceServlet.printMetrics(sessionOutput.out(), System.nanoTime() - elapsedStart,
-                        metadata.getJobDuration(), stats.getCount(), stats.getSize(), metadata.getProcessedObjects(), 0,
-                        0, HttpUtil.getPreferredCharset(request));
-                sessionOutput.out().print("}\n");
+                printer.begin();
+                printer.addResultPrinter(new ResultsPrinter(appCtx, resultReader, null, stats, sessionOutput));
+                printer.printResults();
+                ResponseMertics mertics =
+                        ResponseMertics.of(System.nanoTime() - elapsedStart, metadata.getJobDuration(),
+                                stats.getCount(), stats.getSize(), metadata.getProcessedObjects(), 0, 0);
+                printer.addFooterPrinter(new MetricsPrinter(mertics, HttpUtil.getPreferredCharset(request)));
+                printer.printFooters();
+                printer.end();
             } else {
                 ResultUtil.printResults(appCtx, resultReader, sessionOutput, new Stats(), null);
             }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
index 06b75e3..4eb3524 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
@@ -31,7 +31,6 @@
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.nio.charset.Charset;
-import java.nio.charset.StandardCharsets;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
@@ -43,10 +42,23 @@
 import java.util.function.Function;
 
 import org.apache.asterix.algebra.base.ILangExtension;
+import org.apache.asterix.app.result.ExecutionError;
+import org.apache.asterix.app.result.ResponseMertics;
+import org.apache.asterix.app.result.ResponsePrinter;
+import org.apache.asterix.app.result.fields.ClientContextIdPrinter;
+import org.apache.asterix.app.result.fields.ErrorsPrinter;
+import org.apache.asterix.app.result.fields.MetricsPrinter;
+import org.apache.asterix.app.result.fields.ParseOnlyResultPrinter;
+import org.apache.asterix.app.result.fields.PlansPrinter;
+import org.apache.asterix.app.result.fields.RequestIdPrinter;
+import org.apache.asterix.app.result.fields.SignaturePrinter;
+import org.apache.asterix.app.result.fields.StatusPrinter;
+import org.apache.asterix.app.result.fields.TypePrinter;
+import org.apache.asterix.app.result.fields.WarningsPrinter;
 import org.apache.asterix.app.translator.QueryTranslator;
-import org.apache.asterix.common.api.Duration;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.api.IClusterManagementWork;
+import org.apache.asterix.common.api.ICodedMessage;
 import org.apache.asterix.common.api.IReceptionist;
 import org.apache.asterix.common.api.IRequestReference;
 import org.apache.asterix.common.config.GlobalConfig;
@@ -65,7 +77,6 @@
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.translator.ExecutionPlans;
-import org.apache.asterix.translator.ExecutionPlansJsonPrintUtil;
 import org.apache.asterix.translator.IRequestParameters;
 import org.apache.asterix.translator.IStatementExecutor;
 import org.apache.asterix.translator.IStatementExecutor.ResultDelivery;
@@ -184,26 +195,6 @@
         }
     }
 
-    private enum Metrics {
-        ELAPSED_TIME("elapsedTime"),
-        EXECUTION_TIME("executionTime"),
-        RESULT_COUNT("resultCount"),
-        RESULT_SIZE("resultSize"),
-        ERROR_COUNT("errorCount"),
-        PROCESSED_OBJECTS_COUNT("processedObjects"),
-        WARNING_COUNT("warningCount");
-
-        private final String str;
-
-        Metrics(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
     protected static final class RequestExecutionState {
         private long execStart = -1;
         private long execEnd = -1;
@@ -288,68 +279,6 @@
         SessionOutput.ResultAppender appendStatus = ResultUtil.createResultStatusAppender();
         SessionConfig sessionConfig = new SessionConfig(SessionConfig.OutputFormat.CLEAN_JSON);
         return new SessionOutput(sessionConfig, resultWriter, resultPrefix, resultPostfix, null, appendStatus);
-    }
-
-    private static void printClientContextID(PrintWriter pw, QueryServiceRequestParameters params) {
-        if (params.getClientContextID() != null && !params.getClientContextID().isEmpty()) {
-            ResultUtil.printField(pw, ResultFields.CLIENT_ID.str(), params.getClientContextID());
-        }
-    }
-
-    private static void printSignature(PrintWriter pw, QueryServiceRequestParameters param) {
-        if (param.isSignature()) {
-            pw.print("\t\"");
-            pw.print(ResultFields.SIGNATURE.str());
-            pw.print("\": {\n");
-            pw.print("\t");
-            ResultUtil.printField(pw, "*", "*", false);
-            pw.print("\t},\n");
-        }
-    }
-
-    private static void printType(PrintWriter pw, SessionConfig sessionConfig) {
-        switch (sessionConfig.fmt()) {
-            case ADM:
-                ResultUtil.printField(pw, ResultFields.TYPE.str(), HttpUtil.ContentType.APPLICATION_ADM);
-                break;
-            case CSV:
-                String contentType = HttpUtil.ContentType.CSV + "; header="
-                        + (sessionConfig.is(SessionConfig.FORMAT_CSV_HEADER) ? "present" : "absent");
-                ResultUtil.printField(pw, ResultFields.TYPE.str(), contentType);
-                break;
-            default:
-                break;
-        }
-    }
-
-    public static void printMetrics(PrintWriter pw, long elapsedTime, long executionTime, long resultCount,
-            long resultSize, long processedObjects, long errorCount, long warnCount, Charset resultCharset) {
-        boolean hasErrors = errorCount != 0;
-        boolean hasWarnings = warnCount != 0;
-        boolean useAscii = !StandardCharsets.UTF_8.equals(resultCharset)
-                && !"μ".contentEquals(resultCharset.decode(resultCharset.encode("μ")));
-        pw.print("\t\"");
-        pw.print(ResultFields.METRICS.str());
-        pw.print("\": {\n");
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.ELAPSED_TIME.str(), Duration.formatNanos(elapsedTime, useAscii));
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.EXECUTION_TIME.str(), Duration.formatNanos(executionTime, useAscii));
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.RESULT_COUNT.str(), resultCount, true);
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.RESULT_SIZE.str(), resultSize, true);
-        pw.print("\t");
-        ResultUtil.printField(pw, Metrics.PROCESSED_OBJECTS_COUNT.str(), processedObjects, hasWarnings || hasErrors);
-        if (hasWarnings) {
-            pw.print("\t");
-            ResultUtil.printField(pw, Metrics.WARNING_COUNT.str(), warnCount, hasErrors);
-        }
-        if (hasErrors) {
-            pw.print("\t");
-            ResultUtil.printField(pw, Metrics.ERROR_COUNT.str(), errorCount, false);
-        }
-        pw.print("\t}\n");
     }
 
     protected String getOptText(JsonNode node, Parameter parameter) {
@@ -539,72 +468,94 @@
         long errorCount = 1;
         Stats stats = new Stats();
         RequestExecutionState execution = new RequestExecutionState();
-        List<ExecutionWarning> warnings = Collections.emptyList();
+        List<ICodedMessage> warnings = Collections.emptyList();
         Charset resultCharset = HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
         PrintWriter httpWriter = response.writer();
         SessionOutput sessionOutput = createSessionOutput(httpWriter);
         QueryServiceRequestParameters param = newRequestParameters();
+        ResponsePrinter responsePrinter = new ResponsePrinter(sessionOutput);
+        ResultDelivery delivery = ResultDelivery.IMMEDIATE;
         try {
             // buffer the output until we are ready to set the status of the response message correctly
-            sessionOutput.hold();
-            sessionOutput.out().print("{\n");
+            responsePrinter.begin();
             Map<String, String> optionalParams = null;
             if (optionalParamProvider != null) {
                 optionalParams = optionalParamProvider.apply(request);
             }
             setRequestParam(request, param, optionalParams);
             LOGGER.info(() -> "handleRequest: " + LogRedactionUtil.userData(param.toString()));
-            ResultDelivery delivery = parseResultDelivery(param.getMode());
+            delivery = parseResultDelivery(param.getMode());
             setSessionConfig(sessionOutput, param, delivery);
             final ResultProperties resultProperties = param.getMaxResultReads() == null ? new ResultProperties(delivery)
                     : new ResultProperties(delivery, Long.parseLong(param.getMaxResultReads()));
-            printAdditionalResultFields(sessionOutput.out());
-            printRequestId(sessionOutput.out(), requestRef.getUuid());
-            printClientContextID(sessionOutput.out(), param);
-            if (!param.isParseOnly()) {
-                printSignature(sessionOutput.out(), param);
-            }
-            printType(sessionOutput.out(), sessionOutput.config());
+            buildResponseHeaders(requestRef, sessionOutput, param, responsePrinter, delivery);
+            responsePrinter.printHeaders();
             validateStatement(param.getStatement());
             String statementsText = param.getStatement() + ";";
             if (param.isParseOnly()) {
                 ResultUtil.ParseOnlyResult parseOnlyResult = parseStatement(statementsText);
                 setAccessControlHeaders(request, response);
-                response.setStatus(HttpResponseStatus.OK);
-                printParseOnlyValueResult(sessionOutput, parseOnlyResult);
-                ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
+                response.setStatus(execution.getHttpStatus());
+                responsePrinter.addResultPrinter(new ParseOnlyResultPrinter(parseOnlyResult));
             } else {
                 Map<String, byte[]> statementParams = org.apache.asterix.app.translator.RequestParameters
                         .serializeParameterValues(param.getStatementParams());
                 setAccessControlHeaders(request, response);
                 response.setStatus(execution.getHttpStatus());
                 executeStatement(requestRef, statementsText, sessionOutput, resultProperties, stats, param, execution,
-                        optionalParams, statementParams);
-                if (ResultDelivery.IMMEDIATE == delivery || ResultDelivery.DEFERRED == delivery) {
-                    ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
-                }
-            }
-            if (!warnings.isEmpty()) {
-                printWarnings(sessionOutput.out(), warnings);
+                        optionalParams, statementParams, responsePrinter);
             }
             errorCount = 0;
         } catch (Exception | TokenMgrError | org.apache.asterix.aqlplus.parser.TokenMgrError e) {
             handleExecuteStatementException(e, execution, param);
             response.setStatus(execution.getHttpStatus());
-            printError(sessionOutput.out(), e);
-            ResultUtil.printStatus(sessionOutput, execution.getResultStatus());
+            requestFailed(e, responsePrinter);
         } finally {
-            // make sure that we stop buffering and return the result to the http response
-            sessionOutput.release();
             execution.finish();
         }
-        printMetrics(sessionOutput.out(), System.nanoTime() - elapsedStart, execution.duration(), stats.getCount(),
-                stats.getSize(), stats.getProcessedObjects(), errorCount, warnings.size(), resultCharset);
-        sessionOutput.out().print("}\n");
-        sessionOutput.out().flush();
+        responsePrinter.printResults();
+        buildResponseFooters(elapsedStart, errorCount, stats, execution, warnings, resultCharset, responsePrinter,
+                delivery);
+        responsePrinter.printFooters();
+        responsePrinter.end();
         if (sessionOutput.out().checkError()) {
             LOGGER.warn("Error flushing output writer");
         }
+    }
+
+    protected void buildResponseHeaders(IRequestReference requestRef, SessionOutput sessionOutput,
+            QueryServiceRequestParameters param, ResponsePrinter responsePrinter, ResultDelivery delivery) {
+        responsePrinter.addHeaderPrinter(new RequestIdPrinter(requestRef.getUuid()));
+        if (param.getClientContextID() != null && !param.getClientContextID().isEmpty()) {
+            responsePrinter.addHeaderPrinter(new ClientContextIdPrinter(param.getClientContextID()));
+        }
+        if (param.isSignature() && delivery != ResultDelivery.ASYNC && !param.isParseOnly()) {
+            responsePrinter.addHeaderPrinter(SignaturePrinter.INSTANCE);
+        }
+        if (sessionOutput.config().fmt() == SessionConfig.OutputFormat.ADM
+                || sessionOutput.config().fmt() == SessionConfig.OutputFormat.CSV) {
+            responsePrinter.addHeaderPrinter(new TypePrinter(sessionOutput.config()));
+        }
+    }
+
+    protected void buildResponseResults(ResponsePrinter responsePrinter, SessionOutput sessionOutput,
+            ExecutionPlans plans) {
+        responsePrinter.addResultPrinter(new PlansPrinter(plans, sessionOutput.config().getPlanFormat()));
+    }
+
+    protected void buildResponseFooters(long elapsedStart, long errorCount, Stats stats,
+            RequestExecutionState execution, List<ICodedMessage> warnings, Charset resultCharset,
+            ResponsePrinter responsePrinter, ResultDelivery delivery) {
+        if (ResultDelivery.ASYNC != delivery) {
+            // in case of ASYNC delivery, the status is printed by query translator
+            responsePrinter.addFooterPrinter(new StatusPrinter(execution.getResultStatus()));
+        }
+        if (!warnings.isEmpty()) {
+            responsePrinter.addFooterPrinter(new WarningsPrinter(warnings));
+        }
+        final ResponseMertics mertics = ResponseMertics.of(System.nanoTime() - elapsedStart, execution.duration(),
+                stats.getCount(), stats.getSize(), stats.getProcessedObjects(), errorCount, warnings.size());
+        responsePrinter.addFooterPrinter(new MetricsPrinter(mertics, resultCharset));
     }
 
     protected void validateStatement(String statement) throws RuntimeDataException {
@@ -628,7 +579,8 @@
     protected void executeStatement(IRequestReference requestReference, String statementsText,
             SessionOutput sessionOutput, ResultProperties resultProperties, Stats stats,
             QueryServiceRequestParameters param, RequestExecutionState execution,
-            Map<String, String> optionalParameters, Map<String, byte[]> statementParameters) throws Exception {
+            Map<String, String> optionalParameters, Map<String, byte[]> statementParameters,
+            ResponsePrinter responsePrinter) throws Exception {
         IClusterManagementWork.ClusterState clusterState =
                 ((ICcApplicationContext) appCtx).getClusterStateManager().getState();
         if (clusterState != IClusterManagementWork.ClusterState.ACTIVE) {
@@ -639,7 +591,7 @@
         List<Statement> statements = parser.parse();
         MetadataManager.INSTANCE.init();
         IStatementExecutor translator = statementExecutorFactory.create((ICcApplicationContext) appCtx, statements,
-                sessionOutput, compilationProvider, componentProvider);
+                sessionOutput, compilationProvider, componentProvider, responsePrinter);
         execution.start();
         Map<String, IAObject> stmtParams =
                 org.apache.asterix.app.translator.RequestParameters.deserializeParameterValues(statementParameters);
@@ -648,7 +600,7 @@
                 optionalParameters, stmtParams, param.isMultiStatement());
         translator.compileAndExecute(getHyracksClientConnection(), requestParameters);
         execution.end();
-        printExecutionPlans(sessionOutput, translator.getExecutionPlans());
+        buildResponseResults(responsePrinter, sessionOutput, translator.getExecutionPlans());
     }
 
     protected void handleExecuteStatementException(Throwable t, RequestExecutionState state,
@@ -716,42 +668,9 @@
                 && "present".equals(getParameterValue(param.getFormat(), Attribute.HEADER.str())));
     }
 
-    protected void printError(PrintWriter sessionOut, Throwable throwable) {
-        ResultUtil.printError(sessionOut, throwable);
-    }
-
-    protected void printAdditionalResultFields(PrintWriter sessionOut) {
-        // do nothing
-    }
-
-    protected void printWarnings(PrintWriter pw, List<ExecutionWarning> warnings) {
-        ResultUtil.printWarnings(pw, warnings);
-    }
-
-    protected void printParseOnlyValueResult(SessionOutput output, ResultUtil.ParseOnlyResult parseOnlyResult) {
-        final PrintWriter pw = output.out();
-        pw.print("\t\"");
-        pw.print(ResultFields.RESULTS.str()); //TODO: use ResultUtil, ResultPrinter
-        pw.print("\":");
-        pw.print(parseOnlyResult.asJson());
-        pw.print(",\n");
-    }
-
-    protected void printExecutionPlans(SessionOutput output, ExecutionPlans executionPlans) {
-        final PrintWriter pw = output.out();
-        pw.print("\t\"");
-        pw.print(ResultFields.PLANS.str());
-        pw.print("\":");
-        final SessionConfig.PlanFormat planFormat = output.config().getPlanFormat();
-        switch (planFormat) {
-            case JSON:
-            case STRING:
-                pw.print(ExecutionPlansJsonPrintUtil.asJson(executionPlans, planFormat));
-                break;
-            default:
-                throw new IllegalStateException("Unrecognized plan format: " + planFormat);
-        }
-        pw.print(",\n");
+    protected void requestFailed(Throwable throwable, ResponsePrinter responsePrinter) {
+        final ExecutionError executionError = ExecutionError.of(throwable);
+        responsePrinter.addResultPrinter(new ErrorsPrinter(Collections.singletonList(executionError)));
     }
 
     protected QueryServiceRequestParameters newRequestParameters() {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
index df09aee..c91f0e3 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
@@ -21,12 +21,19 @@
 import static org.apache.asterix.api.http.server.AbstractQueryApiServlet.ResultStatus.FAILED;
 
 import java.io.PrintWriter;
+import java.util.Collections;
 import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 
+import org.apache.asterix.app.result.ExecutionError;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.fields.ErrorsPrinter;
+import org.apache.asterix.app.result.fields.ResultHandlePrinter;
+import org.apache.asterix.app.result.fields.StatusPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.translator.SessionOutput;
 import org.apache.hyracks.api.result.ResultJobRecord;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
@@ -52,37 +59,32 @@
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
         }
-
         ResultReader resultReader = new ResultReader(getResultSet(), handle.getJobId(), handle.getResultSetId());
-
         final ResultJobRecord.Status resultReaderStatus = resultReader.getStatus();
         if (resultReaderStatus == null) {
             LOGGER.log(Level.INFO, "No results for: \"" + strHandle + "\"");
             response.setStatus(HttpResponseStatus.NOT_FOUND);
             return;
         }
-
         ResultStatus resultStatus = resultStatus(resultReaderStatus);
         Exception ex = extractException(resultReaderStatus);
-
         HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, request);
         final PrintWriter resultWriter = response.writer();
-
-        HttpResponseStatus httpStatus = HttpResponseStatus.OK;
-
-        resultWriter.print("{\n");
-        ResultUtil.printStatus(resultWriter, resultStatus, (ex != null) || ResultStatus.SUCCESS == resultStatus);
-
+        response.setStatus(HttpResponseStatus.OK);
+        SessionOutput sessionOutput = new SessionOutput(resultWriter);
+        ResponsePrinter printer = new ResponsePrinter(sessionOutput);
+        printer.begin();
+        printer.addHeaderPrinter(new StatusPrinter(resultStatus));
+        printer.printHeaders();
         if (ResultStatus.SUCCESS == resultStatus) {
             String servletPath = servletPath(request).replace("status", "result");
             String resHandle = "http://" + host(request) + servletPath + strHandle;
-            printHandle(resultWriter, resHandle, false);
+            printer.addResultPrinter(new ResultHandlePrinter(resHandle));
         } else if (ex != null) {
-            ResultUtil.printError(resultWriter, ex, false);
+            printer.addResultPrinter(new ErrorsPrinter(Collections.singletonList(ExecutionError.of(ex))));
         }
-
-        resultWriter.print("}\n");
-        response.setStatus(httpStatus);
+        printer.printResults();
+        printer.end();
         if (response.writer().checkError()) {
             LOGGER.warn("Error flushing output writer");
         }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
index fa3c03d..63da9ce 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ResultUtil.java
@@ -32,11 +32,12 @@
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultPrinter;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.fields.ResultHandlePrinter;
+import org.apache.asterix.app.result.fields.ResultsPrinter;
+import org.apache.asterix.app.result.fields.StatusPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
-import org.apache.asterix.lang.aql.parser.TokenMgrError;
 import org.apache.asterix.lang.common.expression.VariableExpr;
 import org.apache.asterix.lang.sqlpp.util.SqlppVariableUtil;
 import org.apache.asterix.om.types.ARecordType;
@@ -47,8 +48,6 @@
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.api.exceptions.HyracksException;
-import org.apache.hyracks.util.JSONUtil;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
@@ -91,76 +90,6 @@
         new ResultPrinter(appCtx, output, stats, recordType).print(record);
     }
 
-    public static void printResultHandle(SessionOutput output, ResultHandle handle) throws HyracksDataException {
-        try {
-            final AlgebricksAppendable app = new AlgebricksAppendable(output.out());
-            output.appendHandle(app, handle.toString());
-        } catch (AlgebricksException e) {
-            LOGGER.warn("error printing handle", e);
-        }
-    }
-
-    public static void printStatus(SessionOutput output, AbstractQueryApiServlet.ResultStatus rs) {
-        try {
-            final AlgebricksAppendable app = new AlgebricksAppendable(output.out());
-            output.appendStatus(app, rs.str());
-        } catch (AlgebricksException e) {
-            LOGGER.warn("error printing status", e);
-        }
-    }
-
-    public static void printStatus(PrintWriter pw, AbstractQueryApiServlet.ResultStatus rs, boolean comma) {
-        printField(pw, AbstractQueryApiServlet.ResultFields.STATUS.str(), rs.str(), comma);
-    }
-
-    public static void printError(PrintWriter pw, Throwable e) {
-        printError(pw, e, true);
-    }
-
-    public static void printError(PrintWriter pw, Throwable e, boolean comma) {
-        printError(pw, e, 1, comma);
-    }
-
-    public static void printError(PrintWriter pw, Throwable e, int code, boolean comma) {
-        Throwable rootCause = getRootCause(e);
-        String msg = rootCause.getMessage();
-        if (!(rootCause instanceof AlgebricksException || rootCause instanceof HyracksException
-                || rootCause instanceof TokenMgrError
-                || rootCause instanceof org.apache.asterix.aqlplus.parser.TokenMgrError)) {
-            msg = rootCause.getClass().getSimpleName() + (msg == null ? "" : ": " + msg);
-        }
-        printError(pw, msg, code, comma);
-    }
-
-    public static void printError(PrintWriter pw, String msg, int code, boolean comma) {
-        pw.print("\t\"");
-        pw.print(AbstractQueryApiServlet.ResultFields.ERRORS.str());
-        pw.print("\": [{ \n\t");
-        printField(pw, QueryServiceServlet.ErrorField.CODE.str(), code);
-        pw.print("\t");
-        printField(pw, QueryServiceServlet.ErrorField.MSG.str(), JSONUtil.escape(msg), false);
-        pw.print(comma ? "\t}],\n" : "\t}]\n");
-    }
-
-    public static void printWarnings(PrintWriter pw, List<ExecutionWarning> warnings) {
-        pw.print("\t\"");
-        pw.print(AbstractQueryApiServlet.ResultFields.WARNINGS.str());
-        pw.print("\": [");
-        for (int i = 0; i < warnings.size(); i++) {
-            final ExecutionWarning warning = warnings.get(i);
-            pw.print("{ \n\t");
-            printField(pw, QueryServiceServlet.ErrorField.CODE.str(), warning.getCode());
-            pw.print("\t");
-            printField(pw, QueryServiceServlet.ErrorField.MSG.str(), JSONUtil.escape(warning.getMessage()), false);
-            pw.print("\t} \n\t");
-            boolean lastWarning = i == warnings.size() - 1;
-            if (!lastWarning) {
-                pw.print(",");
-            }
-        }
-        pw.print("],\n");
-    }
-
     public static void printField(PrintWriter pw, String name, String value) {
         printField(pw, name, value, true);
     }
@@ -185,7 +114,6 @@
         if (comma) {
             pw.print(',');
         }
-        pw.print('\n');
     }
 
     public static ObjectNode getErrorResponse(int errorCode, String errorMessage, String errorSummary,
@@ -359,7 +287,7 @@
             @Override
             public AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException {
                 app.append("\t\"");
-                app.append(AbstractQueryApiServlet.ResultFields.RESULTS.str());
+                app.append(ResultsPrinter.FIELD_NAME);
                 if (resultNo >= 0) {
                     app.append('-').append(String.valueOf(resultNo));
                 }
@@ -371,17 +299,17 @@
     }
 
     public static SessionOutput.ResultDecorator createPostResultDecorator() {
-        return app -> app.append("\t,\n");
+        return app -> app.append("\t");
     }
 
     public static SessionOutput.ResultAppender createResultHandleAppender(String handleUrl) {
-        return (app, handle) -> app.append("\t\"").append(AbstractQueryApiServlet.ResultFields.HANDLE.str())
-                .append("\": \"").append(handleUrl).append(handle).append("\",\n");
+        return (app, handle) -> app.append("\t\"").append(ResultHandlePrinter.FIELD_NAME).append("\": \"")
+                .append(handleUrl).append(handle).append("\"");
     }
 
     public static SessionOutput.ResultAppender createResultStatusAppender() {
-        return (app, status) -> app.append("\t\"").append(AbstractQueryApiServlet.ResultFields.STATUS.str())
-                .append("\": \"").append(status).append("\",\n");
+        return (app, status) -> app.append("\t\"").append(StatusPrinter.FIELD_NAME).append("\": \"").append(status)
+                .append("\"");
     }
 
     public static class ParseOnlyResult {
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
index a5c8645..1f8e44c 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/java/AsterixJavaClient.java
@@ -25,6 +25,7 @@
 import java.util.UUID;
 
 import org.apache.asterix.api.common.APIFramework;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.api.RequestReference;
 import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -128,7 +129,7 @@
         SessionOutput output = new SessionOutput(conf, writer);
 
         IStatementExecutor translator = statementExecutorFactory.create(appCtx, statements, output, compilationProvider,
-                storageComponentProvider);
+                storageComponentProvider, new ResponsePrinter(output));
         final RequestReference requestReference =
                 RequestReference.of(UUID.randomUUID().toString(), "CC", System.currentTimeMillis());
         final IRequestParameters requestParameters = new RequestParameters(requestReference, statement, null,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
index a89728c..b0b94c5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/message/ExecuteStatementRequestMessage.java
@@ -28,6 +28,7 @@
 import org.apache.asterix.algebra.base.ILangExtension;
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.app.cc.CCExtensionManager;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.translator.RequestParameters;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.api.IRequestReference;
@@ -130,7 +131,7 @@
             IStatementExecutor.ResultMetadata outMetadata = new IStatementExecutor.ResultMetadata();
             MetadataManager.INSTANCE.init();
             IStatementExecutor translator = statementExecutorFactory.create(ccAppCtx, statements, sessionOutput,
-                    compilationProvider, storageComponentProvider);
+                    compilationProvider, storageComponentProvider, new ResponsePrinter(sessionOutput));
             final IStatementExecutor.Stats stats = new IStatementExecutor.Stats();
             Map<String, IAObject> stmtParams = RequestParameters.deserializeParameterValues(statementParameters);
             final IRequestParameters requestParameters =
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java
new file mode 100644
index 0000000..c57e61d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionError.java
@@ -0,0 +1,57 @@
+/*
+ * 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.app.result;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.ICodedMessage;
+import org.apache.asterix.lang.aql.parser.TokenMgrError;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.exceptions.HyracksException;
+
+public class ExecutionError implements ICodedMessage {
+
+    private final int code;
+    private final String message;
+
+    private ExecutionError(int code, String message) {
+        this.code = code;
+        this.message = message;
+    }
+
+    public static ExecutionError of(Throwable t) {
+        Throwable rootCause = ResultUtil.getRootCause(t);
+        String msg = rootCause.getMessage();
+        if (!(rootCause instanceof AlgebricksException || rootCause instanceof HyracksException
+                || rootCause instanceof TokenMgrError
+                || rootCause instanceof org.apache.asterix.aqlplus.parser.TokenMgrError)) {
+            msg = rootCause.getClass().getSimpleName() + (msg == null ? "" : ": " + msg);
+        }
+        return new ExecutionError(1, msg);
+    }
+
+    @Override
+    public int getCode() {
+        return code;
+    }
+
+    @Override
+    public String getMessage() {
+        return message;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
similarity index 86%
rename from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
rename to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
index baaa5bd..29eb098 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ExecutionWarning.java
@@ -16,9 +16,11 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.app.result;
 
-public class ExecutionWarning {
+import org.apache.asterix.common.api.ICodedMessage;
+
+public class ExecutionWarning implements ICodedMessage {
 
     private final int code;
     private final String message;
@@ -28,10 +30,12 @@
         this.message = message;
     }
 
+    @Override
     public int getCode() {
         return code;
     }
 
+    @Override
     public String getMessage() {
         return message;
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java
new file mode 100644
index 0000000..666b759
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponseMertics.java
@@ -0,0 +1,74 @@
+/*
+ * 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.app.result;
+
+public class ResponseMertics {
+
+    private long elapsedTime;
+    private long executionTime;
+    private long resultCount;
+    private long resultSize;
+    private long processedObjects;
+    private long errorCount;
+    private long warnCount;
+
+    private ResponseMertics() {
+    }
+
+    public static ResponseMertics of(long elapsedTime, long executionTime, long resultCount, long resultSize,
+            long processedObjects, long errorCount, long warnCount) {
+        ResponseMertics mertics = new ResponseMertics();
+        mertics.elapsedTime = elapsedTime;
+        mertics.executionTime = executionTime;
+        mertics.resultCount = resultCount;
+        mertics.resultSize = resultSize;
+        mertics.processedObjects = processedObjects;
+        mertics.errorCount = errorCount;
+        mertics.warnCount = warnCount;
+        return mertics;
+    }
+
+    public long getElapsedTime() {
+        return elapsedTime;
+    }
+
+    public long getExecutionTime() {
+        return executionTime;
+    }
+
+    public long getResultCount() {
+        return resultCount;
+    }
+
+    public long getResultSize() {
+        return resultSize;
+    }
+
+    public long getProcessedObjects() {
+        return processedObjects;
+    }
+
+    public long getErrorCount() {
+        return errorCount;
+    }
+
+    public long getWarnCount() {
+        return warnCount;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java
new file mode 100644
index 0000000..43e128d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResponsePrinter.java
@@ -0,0 +1,109 @@
+/*
+ * 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.app.result;
+
+import java.io.PrintWriter;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.common.api.IResponsePrinter;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ResponsePrinter implements IResponsePrinter {
+
+    private final SessionOutput sessionOutput;
+    private final List<IResponseFieldPrinter> headers = new ArrayList<>();
+    private final List<IResponseFieldPrinter> results = new ArrayList<>();
+    private final List<IResponseFieldPrinter> footers = new ArrayList<>();
+    private boolean headersPrinted = false;
+    private boolean resultsPrinted = false;
+
+    public ResponsePrinter(SessionOutput sessionOutput) {
+        this.sessionOutput = sessionOutput;
+    }
+
+    @Override
+    public void begin() {
+        sessionOutput.hold();
+        sessionOutput.out().print("{\n");
+    }
+
+    @Override
+    public void addHeaderPrinter(IResponseFieldPrinter printer) {
+        headers.add(printer);
+    }
+
+    @Override
+    public void addResultPrinter(IResponseFieldPrinter printer) {
+        results.add(printer);
+    }
+
+    @Override
+    public void addFooterPrinter(IResponseFieldPrinter printer) {
+        footers.add(printer);
+    }
+
+    @Override
+    public void printHeaders() throws HyracksDataException {
+        print(headers);
+        headersPrinted = !headers.isEmpty();
+    }
+
+    @Override
+    public void printResults() throws HyracksDataException {
+        sessionOutput.release();
+        print(results);
+        if (!resultsPrinted) {
+            resultsPrinted = !results.isEmpty();
+        }
+        results.clear();
+    }
+
+    @Override
+    public void printFooters() throws HyracksDataException {
+        print(footers);
+    }
+
+    @Override
+    public void end() {
+        sessionOutput.out().print("\n}\n");
+        sessionOutput.release();
+        sessionOutput.out().flush();
+    }
+
+    private void print(List<IResponseFieldPrinter> printers) throws HyracksDataException {
+        final int fieldsCount = printers.size();
+        if ((headersPrinted || resultsPrinted) && fieldsCount > 0) {
+            printFieldSeparator(sessionOutput.out());
+        }
+        for (int i = 0; i < printers.size(); i++) {
+            IResponseFieldPrinter printer = printers.get(i);
+            printer.print(sessionOutput.out());
+            if (i + 1 != fieldsCount) {
+                printFieldSeparator(sessionOutput.out());
+            }
+        }
+    }
+
+    public static void printFieldSeparator(PrintWriter pw) {
+        pw.print(",\n");
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
index 6f9fc47..330cf4b 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
@@ -62,6 +62,6 @@
 
     @Override
     public String toString() {
-        return Long.toString(jobId.getId()) + "-" + Long.toString(resultSetId.getId());
+        return jobId.getId() + "-" + resultSetId.getId();
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java
new file mode 100644
index 0000000..6270c4c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/AbstractCodedMessagePrinter.java
@@ -0,0 +1,71 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+import java.util.List;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.ICodedMessage;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.hyracks.util.JSONUtil;
+
+public abstract class AbstractCodedMessagePrinter implements IResponseFieldPrinter {
+
+    private enum CodedMessageField {
+        CODE("code"),
+        MSG("msg");
+
+        private final String str;
+
+        CodedMessageField(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    private final List<ICodedMessage> messages;
+
+    public AbstractCodedMessagePrinter(List<ICodedMessage> messages) {
+        this.messages = messages;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.print("\t\"");
+        pw.print(getName());
+        pw.print("\": [");
+        for (int i = 0; i < messages.size(); i++) {
+            final ICodedMessage codedMessage = messages.get(i);
+            pw.print("{ \n\t");
+            ResultUtil.printField(pw, CodedMessageField.CODE.str(), codedMessage.getCode());
+            pw.print("\t");
+            ResultUtil.printField(pw, CodedMessageField.MSG.str(), JSONUtil.escape(codedMessage.getMessage()), false);
+            pw.print("\t} \n\t");
+            boolean lastMsg = i == messages.size() - 1;
+            if (!lastMsg) {
+                pw.print(",");
+            }
+        }
+        pw.print("]");
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java
new file mode 100644
index 0000000..c83210d
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ClientContextIdPrinter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class ClientContextIdPrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "clientContextID";
+    private final String clientContextId;
+
+    public ClientContextIdPrinter(String clientContextId) {
+        this.clientContextId = clientContextId;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        ResultUtil.printField(pw, FIELD_NAME, clientContextId, false);
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
similarity index 67%
copy from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
copy to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
index baaa5bd..cecc8e5 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ErrorsPrinter.java
@@ -16,23 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.app.result.fields;
 
-public class ExecutionWarning {
+import java.util.List;
 
-    private final int code;
-    private final String message;
+import org.apache.asterix.common.api.ICodedMessage;
 
-    public ExecutionWarning(int code, String message) {
-        this.code = code;
-        this.message = message;
+public class ErrorsPrinter extends AbstractCodedMessagePrinter {
+
+    private static final String FIELD_NAME = "errors";
+
+    public ErrorsPrinter(List<ICodedMessage> errors) {
+        super(errors);
     }
 
-    public int getCode() {
-        return code;
-    }
-
-    public String getMessage() {
-        return message;
+    @Override
+    public String getName() {
+        return FIELD_NAME;
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ExplainOnlyResultsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ExplainOnlyResultsPrinter.java
new file mode 100644
index 0000000..1949e54
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ExplainOnlyResultsPrinter.java
@@ -0,0 +1,54 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ExplainOnlyResultsPrinter implements IResponseFieldPrinter {
+
+    private final IApplicationContext appCtx;
+    private final String plan;
+    private final SessionOutput sessionOutput;
+
+    public ExplainOnlyResultsPrinter(IApplicationContext appCtx, String plan, SessionOutput sessionOutput) {
+        this.appCtx = appCtx;
+        this.plan = plan;
+        this.sessionOutput = sessionOutput;
+    }
+
+    @Override
+    public void print(PrintWriter pw) throws HyracksDataException {
+        boolean quoteResult = sessionOutput.config().getPlanFormat() == SessionConfig.PlanFormat.STRING;
+        sessionOutput.config().set(SessionConfig.FORMAT_QUOTE_RECORD, quoteResult);
+        ResultUtil.printResults(appCtx, plan, sessionOutput, new IStatementExecutor.Stats(), null);
+    }
+
+    @Override
+    public String getName() {
+        return ResultsPrinter.FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
new file mode 100644
index 0000000..35b5f43
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/MetricsPrinter.java
@@ -0,0 +1,100 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+import java.nio.charset.Charset;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.ResponseMertics;
+import org.apache.asterix.common.api.Duration;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class MetricsPrinter implements IResponseFieldPrinter {
+
+    public enum Metrics {
+        ELAPSED_TIME("elapsedTime"),
+        EXECUTION_TIME("executionTime"),
+        RESULT_COUNT("resultCount"),
+        RESULT_SIZE("resultSize"),
+        ERROR_COUNT("errorCount"),
+        PROCESSED_OBJECTS_COUNT("processedObjects"),
+        WARNING_COUNT("warningCount");
+
+        private final String str;
+
+        Metrics(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
+    public static final String FIELD_NAME = "metrics";
+    private final ResponseMertics mertics;
+    private final Charset resultCharset;
+
+    public MetricsPrinter(ResponseMertics mertics, Charset resultCharset) {
+        this.mertics = mertics;
+        this.resultCharset = resultCharset;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        boolean useAscii = !StandardCharsets.UTF_8.equals(resultCharset)
+                && !"μ".contentEquals(resultCharset.decode(resultCharset.encode("μ")));
+        pw.print("\t\"");
+        pw.print(FIELD_NAME);
+        pw.print("\": {\n");
+        pw.print("\t");
+        ResultUtil.printField(pw, Metrics.ELAPSED_TIME.str(), Duration.formatNanos(mertics.getElapsedTime(), useAscii));
+        pw.print("\n\t");
+        ResultUtil.printField(pw, Metrics.EXECUTION_TIME.str(),
+                Duration.formatNanos(mertics.getExecutionTime(), useAscii));
+        pw.print("\n\t");
+        ResultUtil.printField(pw, Metrics.RESULT_COUNT.str(), mertics.getResultCount(), true);
+        pw.print("\n\t");
+        ResultUtil.printField(pw, Metrics.RESULT_SIZE.str(), mertics.getResultSize(), true);
+        pw.print("\n\t");
+        final boolean hasErrors = mertics.getErrorCount() > 0;
+        final boolean hasWarnings = mertics.getWarnCount() > 0;
+        ResultUtil.printField(pw, Metrics.PROCESSED_OBJECTS_COUNT.str(), mertics.getProcessedObjects(),
+                hasWarnings || hasErrors);
+        pw.print("\n");
+        if (hasWarnings) {
+            pw.print("\t");
+            ResultUtil.printField(pw, Metrics.WARNING_COUNT.str(), mertics.getWarnCount(), hasErrors);
+            pw.print("\n");
+        }
+        if (hasErrors) {
+            pw.print("\t");
+            ResultUtil.printField(pw, Metrics.ERROR_COUNT.str(), mertics.getErrorCount(), false);
+            pw.print("\n");
+        }
+        pw.print("\t}");
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
new file mode 100644
index 0000000..2989d2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/NcResultPrinter.java
@@ -0,0 +1,80 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+import java.util.List;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.message.ExecuteStatementResponseMessage;
+import org.apache.asterix.app.result.ResponsePrinter;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.commons.lang3.tuple.Triple;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.JobId;
+import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.api.result.ResultSetId;
+
+public class NcResultPrinter implements IResponseFieldPrinter {
+
+    private final IStatementExecutor.ResultDelivery delivery;
+    private final ExecuteStatementResponseMessage responseMsg;
+    private final IApplicationContext appCtx;
+    private final IResultSet resultSet;
+    private final SessionOutput sessionOutput;
+
+    public NcResultPrinter(IApplicationContext appCtx, ExecuteStatementResponseMessage responseMsg,
+            IResultSet resultSet, IStatementExecutor.ResultDelivery delivery, SessionOutput sessionOutput) {
+        this.appCtx = appCtx;
+        this.responseMsg = responseMsg;
+        this.delivery = delivery;
+        this.resultSet = resultSet;
+        this.sessionOutput = sessionOutput;
+    }
+
+    @Override
+    public void print(PrintWriter pw) throws HyracksDataException {
+        IStatementExecutor.ResultMetadata resultMetadata = responseMsg.getMetadata();
+        List<Triple<JobId, ResultSetId, ARecordType>> resultSets = resultMetadata.getResultSets();
+        if (delivery == IStatementExecutor.ResultDelivery.IMMEDIATE && !resultSets.isEmpty()) {
+            final IStatementExecutor.Stats stats = responseMsg.getStats();
+            stats.setProcessedObjects(responseMsg.getStats().getProcessedObjects());
+            for (int i = 0; i < resultSets.size(); i++) {
+                Triple<JobId, ResultSetId, ARecordType> rsmd = resultSets.get(i);
+                ResultReader resultReader = new ResultReader(resultSet, rsmd.getLeft(), rsmd.getMiddle());
+                ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, rsmd.getRight());
+                if (i + 1 != resultSets.size()) {
+                    ResponsePrinter.printFieldSeparator(pw);
+                }
+            }
+        } else {
+            pw.append(responseMsg.getResult());
+        }
+    }
+
+    @Override
+    public String getName() {
+        return ResultsPrinter.FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java
new file mode 100644
index 0000000..d7ee269
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ParseOnlyResultPrinter.java
@@ -0,0 +1,46 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class ParseOnlyResultPrinter implements IResponseFieldPrinter {
+
+    private final ResultUtil.ParseOnlyResult parseOnlyResult;
+
+    public ParseOnlyResultPrinter(ResultUtil.ParseOnlyResult parseOnlyResult) {
+        this.parseOnlyResult = parseOnlyResult;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.print("\t\"");
+        pw.print(getName());
+        pw.print("\":");
+        pw.print(parseOnlyResult.asJson());
+    }
+
+    @Override
+    public String getName() {
+        return ResultsPrinter.FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java
new file mode 100644
index 0000000..4021956
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/PlansPrinter.java
@@ -0,0 +1,58 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.translator.ExecutionPlans;
+import org.apache.asterix.translator.ExecutionPlansJsonPrintUtil;
+import org.apache.asterix.translator.SessionConfig;
+
+public class PlansPrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "plans";
+    private final ExecutionPlans executionPlans;
+    private final SessionConfig.PlanFormat planFormat;
+
+    public PlansPrinter(ExecutionPlans executionPlans, SessionConfig.PlanFormat planFormat) {
+        this.executionPlans = executionPlans;
+        this.planFormat = planFormat;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.print("\t\"");
+        pw.print(FIELD_NAME);
+        pw.print("\":");
+        switch (planFormat) {
+            case JSON:
+            case STRING:
+                pw.print(ExecutionPlansJsonPrintUtil.asJson(executionPlans, planFormat));
+                break;
+            default:
+                throw new IllegalStateException("Unrecognized plan format: " + planFormat);
+        }
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java
new file mode 100644
index 0000000..b66cbfb
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/RequestIdPrinter.java
@@ -0,0 +1,45 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class RequestIdPrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "requestID";
+
+    private final String requestId;
+
+    public RequestIdPrinter(String requestId) {
+        this.requestId = requestId;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        ResultUtil.printField(pw, FIELD_NAME, requestId, false);
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java
new file mode 100644
index 0000000..830c751
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultHandlePrinter.java
@@ -0,0 +1,65 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.ResultHandle;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.prettyprint.AlgebricksAppendable;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ResultHandlePrinter implements IResponseFieldPrinter {
+
+    public static final String FIELD_NAME = "handle";
+    private final SessionOutput sessionOutput;
+    private final String handle;
+
+    public ResultHandlePrinter(SessionOutput sessionOutput, ResultHandle handle) {
+        this.sessionOutput = sessionOutput;
+        this.handle = handle.toString();
+    }
+
+    public ResultHandlePrinter(String handle) {
+        this.handle = handle;
+        sessionOutput = null;
+    }
+
+    @Override
+    public void print(PrintWriter pw) throws HyracksDataException {
+        if (sessionOutput != null) {
+            final AlgebricksAppendable app = new AlgebricksAppendable(pw);
+            try {
+                sessionOutput.appendHandle(app, handle);
+            } catch (AlgebricksException e) {
+                throw HyracksDataException.create(e);
+            }
+        } else {
+            ResultUtil.printField(pw, FIELD_NAME, handle, false);
+        }
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
new file mode 100644
index 0000000..52198de
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/ResultsPrinter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.common.api.IApplicationContext;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.om.types.ARecordType;
+import org.apache.asterix.translator.IStatementExecutor;
+import org.apache.asterix.translator.SessionOutput;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class ResultsPrinter implements IResponseFieldPrinter {
+
+    public static final String FIELD_NAME = "results";
+    private final IApplicationContext appCtx;
+    private final ARecordType recordType;
+    private final ResultReader resultReader;
+    private final IStatementExecutor.Stats stats;
+    private final SessionOutput sessionOutput;
+
+    public ResultsPrinter(IApplicationContext appCtx, ResultReader resultReader, ARecordType recordType,
+            IStatementExecutor.Stats stats, SessionOutput sessionOutput) {
+        this.appCtx = appCtx;
+        this.recordType = recordType;
+        this.resultReader = resultReader;
+        this.stats = stats;
+        this.sessionOutput = sessionOutput;
+    }
+
+    @Override
+    public void print(PrintWriter pw) throws HyracksDataException {
+        ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats, recordType);
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
new file mode 100644
index 0000000..fe9d2be
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/SignaturePrinter.java
@@ -0,0 +1,45 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class SignaturePrinter implements IResponseFieldPrinter {
+
+    public static final SignaturePrinter INSTANCE = new SignaturePrinter();
+    private static final String FIELD_NAME = "signature";
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.print("\t\"");
+        pw.print(FIELD_NAME);
+        pw.print("\": {\n");
+        pw.print("\t");
+        ResultUtil.printField(pw, "*", "*", false);
+        pw.print("\n\t}");
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java
new file mode 100644
index 0000000..372ea45
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/StatusPrinter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.app.result.fields;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.AbstractQueryApiServlet;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+
+public class StatusPrinter implements IResponseFieldPrinter {
+
+    public static final String FIELD_NAME = "status";
+    private final AbstractQueryApiServlet.ResultStatus status;
+
+    public StatusPrinter(AbstractQueryApiServlet.ResultStatus status) {
+        this.status = status;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        pw.append("\t\"").append(FIELD_NAME).append("\": \"").append(status.str()).append("\"");
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java
new file mode 100644
index 0000000..788fbcf
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/TypePrinter.java
@@ -0,0 +1,59 @@
+/*
+ * 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.app.result.fields;
+
+import static org.apache.hyracks.http.server.utils.HttpUtil.ContentType.CSV;
+
+import java.io.PrintWriter;
+
+import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.common.api.IResponseFieldPrinter;
+import org.apache.asterix.translator.SessionConfig;
+import org.apache.hyracks.http.server.utils.HttpUtil;
+
+public class TypePrinter implements IResponseFieldPrinter {
+
+    private static final String FIELD_NAME = "type";
+    private final SessionConfig sessionConfig;
+
+    public TypePrinter(SessionConfig sessionConfig) {
+        this.sessionConfig = sessionConfig;
+    }
+
+    @Override
+    public void print(PrintWriter pw) {
+        switch (sessionConfig.fmt()) {
+            case ADM:
+                ResultUtil.printField(pw, FIELD_NAME, HttpUtil.ContentType.APPLICATION_ADM, false);
+                break;
+            case CSV:
+                String contentType =
+                        CSV + "; header=" + (sessionConfig.is(SessionConfig.FORMAT_CSV_HEADER) ? "present" : "absent");
+                ResultUtil.printField(pw, FIELD_NAME, contentType, false);
+                break;
+            default:
+                break;
+        }
+    }
+
+    @Override
+    public String getName() {
+        return FIELD_NAME;
+    }
+}
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
similarity index 66%
copy from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
copy to asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
index baaa5bd..2ebe5f4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/fields/WarningsPrinter.java
@@ -16,23 +16,22 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.app.result.fields;
 
-public class ExecutionWarning {
+import java.util.List;
 
-    private final int code;
-    private final String message;
+import org.apache.asterix.common.api.ICodedMessage;
 
-    public ExecutionWarning(int code, String message) {
-        this.code = code;
-        this.message = message;
+public class WarningsPrinter extends AbstractCodedMessagePrinter {
+
+    private static final String FIELD_NAME = "warnings";
+
+    public WarningsPrinter(List<ICodedMessage> warnings) {
+        super(warnings);
     }
 
-    public int getCode() {
-        return code;
-    }
-
-    public String getMessage() {
-        return message;
+    @Override
+    public String getName() {
+        return FIELD_NAME;
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
index f6bd708..3e47362 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/DefaultStatementExecutorFactory.java
@@ -22,6 +22,7 @@
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -49,7 +50,8 @@
 
     @Override
     public IStatementExecutor create(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
-            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider) {
-        return new QueryTranslator(appCtx, statements, output, compilationProvider, executorService);
+            ILangCompilationProvider compilationProvider, IStorageComponentProvider storageComponentProvider,
+            IResponsePrinter responsePrinter) {
+        return new QueryTranslator(appCtx, statements, output, compilationProvider, executorService, responsePrinter);
     }
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 8c0cc98..60e98f9 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -45,15 +45,20 @@
 import org.apache.asterix.api.common.APIFramework;
 import org.apache.asterix.api.http.server.AbstractQueryApiServlet;
 import org.apache.asterix.api.http.server.ApiServlet;
-import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.app.active.ActiveEntityEventsListener;
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.active.FeedEventsListener;
+import org.apache.asterix.app.result.ExecutionError;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.fields.ErrorsPrinter;
+import org.apache.asterix.app.result.fields.ResultHandlePrinter;
+import org.apache.asterix.app.result.fields.ResultsPrinter;
+import org.apache.asterix.app.result.fields.StatusPrinter;
 import org.apache.asterix.common.api.IClientRequest;
 import org.apache.asterix.common.api.IMetadataLockManager;
 import org.apache.asterix.common.api.IRequestTracker;
+import org.apache.asterix.common.api.IResponsePrinter;
 import org.apache.asterix.common.cluster.IClusterStateManager;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
@@ -224,9 +229,11 @@
     protected final ExecutorService executorService;
     protected final EnumSet<JobFlag> jobFlags = EnumSet.noneOf(JobFlag.class);
     protected final IMetadataLockManager lockManager;
+    protected final IResponsePrinter responsePrinter;
 
     public QueryTranslator(ICcApplicationContext appCtx, List<Statement> statements, SessionOutput output,
-            ILangCompilationProvider compilationProvider, ExecutorService executorService) {
+            ILangCompilationProvider compilationProvider, ExecutorService executorService,
+            IResponsePrinter responsePrinter) {
         this.appCtx = appCtx;
         this.lockManager = appCtx.getMetadataLockManager();
         this.statements = statements;
@@ -238,6 +245,7 @@
         rewriterFactory = compilationProvider.getRewriterFactory();
         activeDataverse = MetadataBuiltinEntities.DEFAULT_DATAVERSE;
         this.executorService = executorService;
+        this.responsePrinter = responsePrinter;
         if (appCtx.getServiceContext().getAppConfig().getBoolean(CCConfig.Option.ENFORCE_FRAME_WRITER_PROTOCOL)) {
             this.jobFlags.add(JobFlag.ENFORCE_CONTRACT);
         }
@@ -1831,8 +1839,8 @@
                     new CompiledLoadFromFileStatement(dataverseName, loadStmt.getDatasetName().getValue(),
                             loadStmt.getAdapter(), loadStmt.getProperties(), loadStmt.dataIsAlreadySorted());
             cls.setSourceLocation(stmt.getSourceLocation());
-            JobSpecification spec =
-                    apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls, null);
+            JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionOutput, cls,
+                    null, responsePrinter);
             afterCompile();
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1963,7 +1971,8 @@
 
         // Query Compilation (happens under the same ongoing metadata transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, (Query) rewrittenResult.first,
-                rewrittenResult.second, stmt == null ? null : stmt.getDatasetName(), sessionOutput, stmt, externalVars);
+                rewrittenResult.second, stmt == null ? null : stmt.getDatasetName(), sessionOutput, stmt, externalVars,
+                responsePrinter);
     }
 
     private JobSpecification rewriteCompileInsertUpsert(IClusterInfoCollector clusterInfoCollector,
@@ -2001,7 +2010,7 @@
         // Insert/upsert statement compilation (happens under the same ongoing metadata
         // transaction)
         return apiFramework.compileQuery(clusterInfoCollector, metadataProvider, rewrittenInsertUpsert.getQuery(),
-                rewrittenResult.second, datasetName, sessionOutput, clfrqs, externalVars);
+                rewrittenResult.second, datasetName, sessionOutput, clfrqs, externalVars, responsePrinter);
     }
 
     protected void handleCreateFeedStatement(MetadataProvider metadataProvider, Statement stmt) throws Exception {
@@ -2506,16 +2515,17 @@
                 createAndRunJob(hcc, jobFlags, null, compiler, locker, resultDelivery, id -> {
                     final ResultReader resultReader = new ResultReader(resultSet, id, resultSetId);
                     updateJobStats(id, stats, metadataProvider.getResultSetId());
-                    // stop buffering and allow for streaming result delivery
-                    sessionOutput.release();
-                    ResultUtil.printResults(appCtx, resultReader, sessionOutput, stats,
-                            metadataProvider.findOutputRecordType());
+                    responsePrinter.addResultPrinter(new ResultsPrinter(appCtx, resultReader,
+                            metadataProvider.findOutputRecordType(), stats, sessionOutput));
+                    responsePrinter.printResults();
                 }, requestParameters, cancellable, appCtx, metadataProvider);
                 break;
             case DEFERRED:
                 createAndRunJob(hcc, jobFlags, null, compiler, locker, resultDelivery, id -> {
                     updateJobStats(id, stats, metadataProvider.getResultSetId());
-                    ResultUtil.printResultHandle(sessionOutput, new ResultHandle(id, resultSetId));
+                    responsePrinter.addResultPrinter(
+                            new ResultHandlePrinter(sessionOutput, new ResultHandle(id, resultSetId)));
+                    responsePrinter.printResults();
                     if (outMetadata != null) {
                         outMetadata.getResultSets()
                                 .add(Triple.of(id, resultSetId, metadataProvider.findOutputRecordType()));
@@ -2543,8 +2553,9 @@
         try {
             createAndRunJob(hcc, jobFlags, jobId, compiler, locker, resultDelivery, id -> {
                 final ResultHandle handle = new ResultHandle(id, resultSetId);
-                ResultUtil.printStatus(sessionOutput, AbstractQueryApiServlet.ResultStatus.RUNNING);
-                ResultUtil.printResultHandle(sessionOutput, handle);
+                responsePrinter.addResultPrinter(new StatusPrinter(AbstractQueryApiServlet.ResultStatus.RUNNING));
+                responsePrinter.addResultPrinter(new ResultHandlePrinter(sessionOutput, handle));
+                responsePrinter.printResults();
                 synchronized (printed) {
                     printed.setTrue();
                     printed.notify();
@@ -2553,8 +2564,13 @@
         } catch (Exception e) {
             if (Objects.equals(JobId.INVALID, jobId.getValue())) {
                 // compilation failed
-                ResultUtil.printStatus(sessionOutput, AbstractQueryApiServlet.ResultStatus.FAILED);
-                ResultUtil.printError(sessionOutput.out(), e);
+                responsePrinter.addResultPrinter(new StatusPrinter(AbstractQueryApiServlet.ResultStatus.FAILED));
+                responsePrinter.addResultPrinter(new ErrorsPrinter(Collections.singletonList(ExecutionError.of(e))));
+                try {
+                    responsePrinter.printResults();
+                } catch (HyracksDataException ex) {
+                    LOGGER.error("failed to print result", ex);
+                }
             } else {
                 GlobalConfig.ASTERIX_LOGGER.log(Level.ERROR,
                         resultDelivery.name() + " job with id " + jobId.getValue() + " " + "failed", e);
@@ -2892,6 +2908,11 @@
         return apiFramework.getExecutionPlans();
     }
 
+    @Override
+    public IResponsePrinter getResponsePrinter() {
+        return responsePrinter;
+    }
+
     public String getActiveDataverse(Identifier dataverse) {
         return getActiveDataverseName(dataverse != null ? dataverse.getValue() : null);
     }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
index debcd19..b74f4c6 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/utils/FeedOperations.java
@@ -29,6 +29,7 @@
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
 import org.apache.asterix.app.translator.QueryTranslator;
 import org.apache.asterix.common.cluster.IClusterStateManager;
@@ -438,7 +439,7 @@
         List<Statement> stmts = new ArrayList<>();
         DefaultStatementExecutorFactory qtFactory = new DefaultStatementExecutorFactory();
         IStatementExecutor translator = qtFactory.create(metadataProvider.getApplicationContext(), stmts, sessionOutput,
-                new SqlppCompilationProvider(), new StorageComponentProvider());
+                new SqlppCompilationProvider(), new StorageComponentProvider(), new ResponsePrinter(sessionOutput));
         return translator;
     }
 
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
index 02bbcf5..fe64dd1 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/app/result/ResultPrinterTest.java
@@ -23,9 +23,11 @@
 import java.io.ByteArrayOutputStream;
 import java.io.PrintWriter;
 import java.nio.charset.StandardCharsets;
+import java.util.Collections;
 
-import org.apache.asterix.api.http.server.AbstractQueryApiServlet;
 import org.apache.asterix.api.http.server.ResultUtil;
+import org.apache.asterix.app.result.fields.ErrorsPrinter;
+import org.apache.asterix.app.result.fields.MetricsPrinter;
 import org.apache.asterix.common.api.IApplicationContext;
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.test.common.ResultExtractor;
@@ -65,7 +67,8 @@
         try {
             rs.print(resultReader);
         } catch (RuntimeException e) {
-            ResultUtil.printError(out, e, true);
+            final ExecutionError error = ExecutionError.of(e);
+            new ErrorsPrinter(Collections.singletonList(error)).print(out);
             printMetrics(out, 1);
         }
         out.print("}");
@@ -98,7 +101,7 @@
 
     private static void printMetrics(PrintWriter pw, long errorCount) {
         pw.print("\t\"");
-        pw.print(AbstractQueryApiServlet.ResultFields.METRICS.str());
+        pw.print(MetricsPrinter.FIELD_NAME);
         pw.print("\": {\n");
         ResultUtil.printField(pw, "errorCount", errorCount, false);
         pw.print("\t}\n");
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
index dfb92b7..80dde8a 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/active/ActiveStatsTest.java
@@ -36,6 +36,7 @@
 import org.apache.asterix.app.active.ActiveNotificationHandler;
 import org.apache.asterix.app.cc.CCExtensionManager;
 import org.apache.asterix.app.nc.NCAppRuntimeContext;
+import org.apache.asterix.app.result.ResponsePrinter;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.external.feed.watch.WaitForStateSubscriber;
 import org.apache.asterix.external.operators.FeedIntakeOperatorNodePushable;
@@ -94,10 +95,12 @@
 
         // Mock MetadataProvider
         CCExtensionManager extensionManager = (CCExtensionManager) appCtx.getExtensionManager();
+        SessionOutput sessionOutput = Mockito.mock(SessionOutput.class);
         IStatementExecutor statementExecutor = extensionManager
                 .getStatementExecutorFactory(appCtx.getServiceContext().getControllerService().getExecutor())
-                .create(appCtx, Collections.emptyList(), Mockito.mock(SessionOutput.class),
-                        extensionManager.getCompilationProvider(Language.SQLPP), appCtx.getStorageComponentProvider());
+                .create(appCtx, Collections.emptyList(), sessionOutput,
+                        extensionManager.getCompilationProvider(Language.SQLPP), appCtx.getStorageComponentProvider(),
+                        new ResponsePrinter(sessionOutput));
         MetadataProvider mdProvider = new MetadataProvider(appCtx, null);
         // Add event listener
         ActiveEntityEventsListener eventsListener = new DummyFeedEventsListener(statementExecutor, appCtx, null,
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
similarity index 67%
copy from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
index baaa5bd..06202c4 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/ICodedMessage.java
@@ -16,23 +16,21 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.common.api;
 
-public class ExecutionWarning {
+public interface ICodedMessage {
 
-    private final int code;
-    private final String message;
+    /**
+     * Gets the code of the message
+     *
+     * @return the code
+     */
+    int getCode();
 
-    public ExecutionWarning(int code, String message) {
-        this.code = code;
-        this.message = message;
-    }
-
-    public int getCode() {
-        return code;
-    }
-
-    public String getMessage() {
-        return message;
-    }
+    /**
+     * Gets the message
+     *
+     * @return the message
+     */
+    String getMessage();
 }
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
similarity index 63%
copy from asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
copy to asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
index baaa5bd..d2006cb 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/ExecutionWarning.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponseFieldPrinter.java
@@ -16,23 +16,26 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.asterix.api.http.server;
+package org.apache.asterix.common.api;
 
-public class ExecutionWarning {
+import java.io.PrintWriter;
 
-    private final int code;
-    private final String message;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-    public ExecutionWarning(int code, String message) {
-        this.code = code;
-        this.message = message;
-    }
+public interface IResponseFieldPrinter {
 
-    public int getCode() {
-        return code;
-    }
+    /**
+     * Prints this field using {@code pw}
+     *
+     * @param pw
+     * @throws HyracksDataException
+     */
+    void print(PrintWriter pw) throws HyracksDataException;
 
-    public String getMessage() {
-        return message;
-    }
+    /**
+     * Gets the name of this field
+     *
+     * @return the name of the field
+     */
+    String getName();
 }
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java
new file mode 100644
index 0000000..a4dc7b5
--- /dev/null
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/api/IResponsePrinter.java
@@ -0,0 +1,76 @@
+/*
+ * 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.common.api;
+
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public interface IResponsePrinter {
+
+    /**
+     * Performs any operations required before printing the response fields.
+     */
+    void begin();
+
+    /**
+     * Adds a response header printer.
+     *
+     * @param printer
+     */
+    void addHeaderPrinter(IResponseFieldPrinter printer);
+
+    /**
+     * Adds a response result printer.
+     *
+     * @param printer
+     */
+    void addResultPrinter(IResponseFieldPrinter printer);
+
+    /**
+     * Adds a response footer printer.
+     *
+     * @param printer
+     */
+    void addFooterPrinter(IResponseFieldPrinter printer);
+
+    /**
+     * Prints the added response headers.
+     *
+     * @throws HyracksDataException
+     */
+    void printHeaders() throws HyracksDataException;
+
+    /**
+     * Prints the added response results.
+     *
+     * @throws HyracksDataException
+     */
+    void printResults() throws HyracksDataException;
+
+    /**
+     * Prints the added response footers.
+     *
+     * @throws HyracksDataException
+     */
+    void printFooters() throws HyracksDataException;
+
+    /**
+     * Performs any operations required after printing the response fields.
+     */
+    void end();
+}

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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: merged
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 4
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/3837/ (6/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:55 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1193/ (2/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:53 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-cancellation-test/5845/ (11/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:50 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://cbjenkins.page.link/HQD68R4mS2s6xFnc8 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:00:28 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-openjdk11/1195/ (16/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:36:02 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/9786/ (15/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:57 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Anon. E. Moose (Code Review)" <de...@asterixdb.apache.org>.
Anon. E. Moose (1000171) has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3: Contrib+1

Analytics Compatibility Tests Successful
https://cbjenkins.page.link/JyuBbn4iC4sJY9bR9 : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Comment-Date: Tue, 28 May 2019 01:44:48 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-spidersilk-tests/768/ (9/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:47 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2: Contrib-2

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4452/ : FAILURE


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:09:13 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-integration-tests/8606/ : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 04:49:09 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3: Contrib+1

BAD Compatibility Tests Successful

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4453/ : SUCCESS


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:55:45 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

BAD Compatibility Tests Started https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/4451/


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:38:17 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ssl-compression/608/ (13/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:24 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/1005/ (8/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:14 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/11317/ (11/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:35:02 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-stabilization-f69489-compat/1103/ (4/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 2
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 02:56:09 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-txnlog/1004/ (5/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 1
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 00:34:54 +0000
Gerrit-HasComments: No

Change in asterixdb[master]: [NO ISSUE][OTH] Introduce ResponsePrinter

Posted by "Jenkins (Code Review)" <de...@asterixdb.apache.org>.
Jenkins has posted comments on this change. ( https://asterix-gerrit.ics.uci.edu/3410 )

Change subject: [NO ISSUE][OTH] Introduce ResponsePrinter
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/5827/ (7/16)


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

Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-MessageType: comment
Gerrit-Change-Id: I797e6615a72c886391fed26281fc648b38fa748f
Gerrit-Change-Number: 3410
Gerrit-PatchSet: 3
Gerrit-Owner: Murtadha Hubail <mh...@apache.org>
Gerrit-Reviewer: Anon. E. Moose (1000171)
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Comment-Date: Mon, 27 May 2019 03:35:44 +0000
Gerrit-HasComments: No