You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by ti...@apache.org on 2017/03/14 05:32:01 UTC

[2/2] asterixdb git commit: Deferred result delivery improvements

Deferred result delivery improvements

- Change representation for result set handles to not require URL encoding
- Report errors from the QueryStatusApiServlet for failed jobs
- Fix repeated result consumption for async/deferred requests
- Better error handling in QueryResultApiServlet
- Add host + path to RequestParameters to provide URLs as handles
- Fix logging for failed async jobs
- Fix test framework to not share request parameters between tests
- Move more shared code to AbstractQueryApiServlet
- Make DatasetClientContext a static inner class of HyracksDataset

Change-Id: I88fe289fe9109ea012c63d82af0083dce6bde31b
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1548
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Michael Blow <mb...@apache.org>
BAD: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>


Project: http://git-wip-us.apache.org/repos/asf/asterixdb/repo
Commit: http://git-wip-us.apache.org/repos/asf/asterixdb/commit/d8a6c71d
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/d8a6c71d
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/d8a6c71d

Branch: refs/heads/master
Commit: d8a6c71d51c46025476a23df263e260fdacc7c15
Parents: 9d69d1b
Author: Till Westmann <ti...@apache.org>
Authored: Mon Mar 13 17:58:16 2017 -0700
Committer: Till Westmann <ti...@apache.org>
Committed: Mon Mar 13 22:31:37 2017 -0700

----------------------------------------------------------------------
 .../http/server/AbstractQueryApiServlet.java    |  52 ++++++++-
 .../api/http/server/QueryResultApiServlet.java  |  46 ++++++--
 .../api/http/server/QueryServiceServlet.java    | 103 ++++++++--------
 .../api/http/server/QueryStatusApiServlet.java  |  54 +++++----
 .../asterix/api/http/server/RestApiServlet.java |   4 +-
 .../apache/asterix/app/result/ResultHandle.java |  47 +++++++-
 .../asterix/app/translator/QueryTranslator.java |  27 +++--
 .../test/common/CancellationTestExecutor.java   |   9 +-
 .../asterix/test/common/ResultExtractor.java    |  17 ++-
 .../asterix/test/common/TestExecutor.java       | 116 +++++++++++++------
 .../query_result_1/query_result_1.1.get.http    |   2 +-
 .../query_result_2/query_result_2.1.get.http    |   2 +-
 .../query_result_3/query_result_3.1.get.http    |   2 +-
 .../query_result_4/query_result_4.1.get.http    |   2 +-
 .../query_status_1/query_status_1.1.get.http    |   2 +-
 .../query_status_2/query_status_2.1.get.http    |   2 +-
 .../query_status_3/query_status_3.1.get.http    |   2 +-
 .../query_status_4/query_status_4.1.get.http    |   2 +-
 .../queries/flwor/at00/at00.5.get.http          |   2 +-
 .../queries/flwor/at00/at00.7.pollget.http      |   2 +-
 .../queries/flwor/at00/at00.8.get.http          |   2 +-
 .../async-failed/async-failed.1.async.sqlpp     |   2 +-
 .../async-failed/async-failed.2.pollget.http    |  22 ----
 .../async-failed/async-failed.2.pollget.uri     |  23 ++++
 .../async-failed/async-failed.3.get.http        |  20 ----
 .../async-repeated/async-repeated.1.async.sqlpp |  22 ++++
 .../async-repeated/async-repeated.2.pollget.uri |  23 ++++
 .../async-repeated/async-repeated.3.get.uri     |  20 ++++
 .../async-repeated/async-repeated.4.get.uri     |  20 ++++
 .../async-repeated/async-repeated.5.query.sqlpp |  20 ++++
 .../async-running/async-running.1.async.sqlpp   |   4 +-
 .../async-running/async-running.2.pollget.http  |  22 ----
 .../async-running/async-running.2.pollget.uri   |  22 ++++
 .../async-running/async-running.3.pollget.http  |  22 ----
 .../async-running/async-running.3.pollget.uri   |  23 ++++
 .../async-running/async-running.4.get.http      |  20 ----
 .../async-running/async-running.4.get.uri       |  20 ++++
 .../async-deferred/async/async.1.async.sqlpp    |   2 +-
 .../async-deferred/async/async.2.pollget.http   |  22 ----
 .../async-deferred/async/async.2.pollget.uri    |  23 ++++
 .../async-deferred/async/async.3.get.http       |  20 ----
 .../async-deferred/async/async.3.get.uri        |  20 ++++
 .../deferred/deferred.1.deferred.sqlpp          |   2 +-
 .../async-deferred/deferred/deferred.2.get.http |  20 ----
 .../async-deferred/deferred/deferred.2.get.uri  |  20 ++++
 .../flwor/at00/at00.4.deferred.sqlpp            |   2 +-
 .../queries_sqlpp/flwor/at00/at00.5.get.http    |  20 ----
 .../queries_sqlpp/flwor/at00/at00.5.get.uri     |  20 ++++
 .../queries_sqlpp/flwor/at00/at00.6.async.sqlpp |   2 +-
 .../flwor/at00/at00.7.pollget.http              |  22 ----
 .../queries_sqlpp/flwor/at00/at00.7.pollget.uri |  23 ++++
 .../queries_sqlpp/flwor/at00/at00.8.get.http    |  20 ----
 .../queries_sqlpp/flwor/at00/at00.8.get.uri     |  20 ++++
 .../async-failed/async-failed.2.json            |   3 -
 .../async-failed/async-failed.2.regex           |   2 +
 .../async-repeated/async-repeated.1.ignore      |   0
 .../async-repeated/async-repeated.2.regex       |   2 +
 .../async-repeated/async-repeated.3.json        |  10 ++
 .../async-repeated/async-repeated.4.json        |  10 ++
 .../async-repeated/async-repeated.5.json        |  10 ++
 .../async-running/async-running.2.json          |   2 +-
 .../resources/runtimets/testsuite_sqlpp.xml     |   7 +-
 .../apache/asterix/common/utils/Servlets.java   |   4 +-
 .../hyracks/api/dataset/DatasetJobRecord.java   |  85 ++++++++++----
 .../api/dataset/IDatasetPartitionManager.java   |   2 -
 .../hyracks/api/exceptions/ErrorCode.java       |   1 +
 .../src/main/resources/errormsg/en.properties   |   1 +
 .../client/dataset/DatasetClientContext.java    |  38 ------
 .../hyracks/client/dataset/HyracksDataset.java  |  22 +++-
 .../client/dataset/HyracksDatasetReader.java    |   9 +-
 .../cc/dataset/DatasetDirectoryService.java     |  39 +++++--
 .../hyracks-control/hyracks-control-nc/pom.xml  |   8 ++
 .../nc/dataset/DatasetPartitionManager.java     |  42 +++----
 .../hyracks/control/nc/dataset/ResultState.java |  23 ++++
 .../result/ResultWriterOperatorDescriptor.java  |  13 ++-
 .../hyracks/http/server/AbstractServlet.java    |   2 +-
 76 files changed, 880 insertions(+), 515 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/AbstractQueryApiServlet.java
----------------------------------------------------------------------
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 f156de5..831244c 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
@@ -29,12 +29,13 @@ import java.util.logging.Level;
 import java.util.logging.Logger;
 
 import org.apache.asterix.app.result.ResultReader;
+import org.apache.asterix.app.result.ResultUtil;
 import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.common.utils.JSONUtil;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.dataset.IHyracksDataset;
 import org.apache.hyracks.client.dataset.HyracksDataset;
-import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.server.AbstractServlet;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
@@ -83,6 +84,22 @@ class AbstractQueryApiServlet extends AbstractServlet {
         }
     }
 
+    public enum ErrorField {
+        CODE("code"),
+        MSG("msg"),
+        STACK("stack");
+
+        private final String str;
+
+        ErrorField(String str) {
+            this.str = str;
+        }
+
+        public String str() {
+            return str;
+        }
+    }
+
     AbstractQueryApiServlet(ConcurrentMap<String, Object> ctx, String[] paths) {
         super(ctx, paths);
     }
@@ -133,11 +150,40 @@ class AbstractQueryApiServlet extends AbstractServlet {
     }
 
     protected static void printStatus(PrintWriter pw, ResultStatus rs) {
-        printField(pw, ResultFields.STATUS.str(), rs.str());
+        printStatus(pw, rs, true);
+    }
+
+    protected static void printStatus(PrintWriter pw, ResultStatus rs, boolean comma) {
+        printField(pw, ResultFields.STATUS.str(), rs.str(), comma);
     }
 
     protected static void printHandle(PrintWriter pw, String handle) {
-        printField(pw, ResultFields.HANDLE.str(), handle);
+        printField(pw, ResultFields.HANDLE.str(), handle, true);
+    }
+
+    protected static void printHandle(PrintWriter pw, String handle, boolean comma) {
+        printField(pw, ResultFields.HANDLE.str(), handle, comma);
+    }
+
+    protected static void printError(PrintWriter pw, Throwable e) throws JsonProcessingException {
+        printError(pw, e, true);
+    }
+
+    protected static void printError(PrintWriter pw, Throwable e, boolean comma) throws JsonProcessingException {
+        Throwable rootCause = ResultUtil.getRootCause(e);
+        if (rootCause == null) {
+            rootCause = e;
+        }
+        final boolean addStack = false;
+        pw.print("\t\"");
+        pw.print(ResultFields.ERRORS.str());
+        pw.print("\": [{ \n");
+        printField(pw, QueryServiceServlet.ErrorField.CODE.str(), "1");
+        final String msg = rootCause.getMessage();
+        printField(pw, QueryServiceServlet.ErrorField.MSG.str(), JSONUtil
+                        .escape(msg != null ? msg : rootCause.getClass().getSimpleName()),
+                addStack);
+        pw.print(comma ? "\t}],\n" : "\t}]\n");
     }
 
     protected static void printField(PrintWriter pw, String name, String value) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryResultApiServlet.java
----------------------------------------------------------------------
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 292dd2a..bfc67cf 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
@@ -23,10 +23,12 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.app.result.ResultUtil;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
+import org.apache.hyracks.api.dataset.DatasetJobRecord;
 import org.apache.hyracks.api.dataset.IHyracksDataset;
 import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.exceptions.ErrorCode;
@@ -50,23 +52,46 @@ public class QueryResultApiServlet extends AbstractQueryApiServlet {
 
     @Override
     protected void get(IServletRequest request, IServletResponse response) throws Exception {
-        response.setStatus(HttpResponseStatus.OK);
         // TODO this seems wrong ...
         HttpUtil.setContentType(response, HttpUtil.ContentType.TEXT_HTML, HttpUtil.Encoding.UTF8);
-        String strHandle = request.getParameter("handle");
         PrintWriter out = response.writer();
 
+        final String strHandle = localPath(request);
+        final ResultHandle handle = ResultHandle.parse(strHandle);
+        if (handle == null) {
+            response.setStatus(HttpResponseStatus.BAD_REQUEST);
+            return;
+        }
+
+        IHyracksDataset hds = getHyracksDataset();
+        ResultReader resultReader = new ResultReader(hds, handle.getJobId(), handle.getResultSetId());
+
+
         try {
-            JsonNode handle = parseHandle(new ObjectMapper(), strHandle, LOGGER);
-            if (handle == null) {
-                response.setStatus(HttpResponseStatus.BAD_REQUEST);
+            DatasetJobRecord.Status status = resultReader.getStatus();
+
+            final HttpResponseStatus httpStatus;
+            if (status == null) {
+                httpStatus = HttpResponseStatus.NOT_FOUND;
+            } else {
+                switch (status.getState()) {
+                    case SUCCESS:
+                        httpStatus = HttpResponseStatus.OK;
+                        break;
+                    case RUNNING:
+                    case IDLE:
+                    case FAILED:
+                        httpStatus = HttpResponseStatus.NOT_FOUND;
+                        break;
+                    default:
+                        httpStatus = HttpResponseStatus.INTERNAL_SERVER_ERROR;
+                        break;
+                }
+            }
+            response.setStatus(httpStatus);
+            if (httpStatus != HttpResponseStatus.OK) {
                 return;
             }
-            JobId jobId = new JobId(handle.get(0).asLong());
-            ResultSetId rsId = new ResultSetId(handle.get(1).asLong());
-
-            IHyracksDataset hds = getHyracksDataset();
-            ResultReader resultReader = new ResultReader(hds, jobId, rsId);
 
             // QQQ The output format is determined by the initial
             // query and cannot be modified here, so calling back to
@@ -94,4 +119,5 @@ public class QueryResultApiServlet extends AbstractQueryApiServlet {
             LOGGER.warning("Error flushing output writer for \"" + strHandle + "\"");
         }
     }
+
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryServiceServlet.java
----------------------------------------------------------------------
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 42bb4f9..67aa914 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.util.logging.Logger;
 
 import org.apache.asterix.api.http.ctx.StatementExecutorContext;
 import org.apache.asterix.api.http.servlet.ServletConstants;
-import org.apache.asterix.app.result.ResultUtil;
 import org.apache.asterix.common.api.IClusterManagementWork;
 import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.context.IStorageComponentProvider;
@@ -56,9 +55,11 @@ import org.apache.hyracks.http.server.utils.HttpUtil;
 
 import com.fasterxml.jackson.core.JsonParseException;
 import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.util.MinimalPrettyPrinter;
 import com.fasterxml.jackson.databind.JsonMappingException;
 import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.node.ObjectNode;
 
 import io.netty.handler.codec.http.HttpHeaderNames;
 import io.netty.handler.codec.http.HttpResponseStatus;
@@ -124,22 +125,6 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
         }
     }
 
-    private enum ErrorField {
-        CODE("code"),
-        MSG("msg"),
-        STACK("stack");
-
-        private final String str;
-
-        ErrorField(String str) {
-            this.str = str;
-        }
-
-        public String str() {
-            return str;
-        }
-    }
-
     private enum Metrics {
         ELAPSED_TIME("elapsedTime"),
         EXECUTION_TIME("executionTime"),
@@ -186,28 +171,30 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
     }
 
     static class RequestParameters {
+        String host;
+        String path;
         String statement;
         String format;
         boolean pretty;
         String clientContextID;
         String mode;
 
+
         @Override
         public String toString() {
-            return append(new StringBuilder()).toString();
-        }
-
-        public StringBuilder append(final StringBuilder sb) {
-            sb.append("{ ");
-            sb.append("\"statement\": \"");
-            JSONUtil.escape(sb, statement);
-            sb.append("\", ");
-            sb.append("\"format\": \"").append(format).append("\", ");
-            sb.append("\"pretty\": ").append(pretty).append(", ");
-            sb.append("\"mode\": ").append(mode).append(", ");
-            sb.append("\"clientContextID\": \"").append(clientContextID).append("\" ");
-            sb.append('}');
-            return sb;
+            try {
+                ObjectMapper om = new ObjectMapper();
+                ObjectNode on = om.createObjectNode();
+                on.put("host", host);
+                on.put("path", path);
+                on.put("statement", JSONUtil.escape(new StringBuilder(), statement).toString());
+                on.put("pretty", pretty);
+                on.put("mode", mode);
+                on.put("clientContextID", clientContextID);
+                return om.writer(new MinimalPrettyPrinter()).writeValueAsString(on);
+            } catch (JsonProcessingException e) { // NOSONAR
+                return e.getMessage();
+            }
         }
     }
 
@@ -249,7 +236,8 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
         return SessionConfig.OutputFormat.CLEAN_JSON;
     }
 
-    private static SessionConfig createSessionConfig(RequestParameters param, PrintWriter resultWriter) {
+    private static SessionConfig createSessionConfig(RequestParameters param, String handleUrl, PrintWriter
+            resultWriter) {
         SessionConfig.ResultDecorator resultPrefix = new SessionConfig.ResultDecorator() {
             int resultNo = -1;
 
@@ -268,8 +256,8 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
 
         SessionConfig.ResultDecorator resultPostfix = app -> app.append("\t,\n");
         SessionConfig.ResultDecorator handlePrefix =
-                app -> app.append("\t\"").append(ResultFields.HANDLE.str()).append("\": ");
-        SessionConfig.ResultDecorator handlePostfix = app -> app.append(",\n");
+                app -> app.append("\t\"").append(ResultFields.HANDLE.str()).append("\": \"").append(handleUrl);
+        SessionConfig.ResultDecorator handlePostfix = app -> app.append("\",\n");
 
         SessionConfig.OutputFormat format = getFormat(param.format);
         SessionConfig sessionConfig =
@@ -308,22 +296,6 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
         }
     }
 
-    private static void printError(PrintWriter pw, Throwable e) throws JsonProcessingException {
-        Throwable rootCause = ResultUtil.getRootCause(e);
-        if (rootCause == null) {
-            rootCause = e;
-        }
-        final boolean addStack = false;
-        pw.print("\t\"");
-        pw.print(ResultFields.ERRORS.str());
-        pw.print("\": [{ \n");
-        printField(pw, ErrorField.CODE.str(), "1");
-        final String msg = rootCause.getMessage();
-        printField(pw, ErrorField.MSG.str(), JSONUtil.escape(msg != null ? msg : rootCause.getClass().getSimpleName()),
-                addStack);
-        pw.print("\t}],\n");
-    }
-
     private static void printMetrics(PrintWriter pw, long elapsedTime, long executionTime, long resultCount,
             long resultSize) {
         pw.print("\t\"");
@@ -355,6 +327,8 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
         int sep = contentTypeParam.indexOf(';');
         final String contentType = sep < 0 ? contentTypeParam.trim() : contentTypeParam.substring(0, sep).trim();
         RequestParameters param = new RequestParameters();
+        param.host = host(request);
+        param.path = servletPath(request);
         if (HttpUtil.ContentType.APPLICATION_JSON.equals(contentType)) {
             try {
                 JsonNode jsonRequest = new ObjectMapper().readTree(getRequestBody(request));
@@ -394,6 +368,32 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
         }
     }
 
+    private static String handlePath(ResultDelivery delivery) {
+        switch (delivery) {
+            case ASYNC:
+                return "/status/";
+            case DEFERRED:
+                return "/result/";
+            case IMMEDIATE:
+            default:
+                return "";
+        }
+    }
+
+    /**
+     * Determines the URL for a result handle based on the host and the path of the incoming request and the result
+     * delivery mode. Usually there will be a "status" endpoint for ASYNC requests that exposes the status of the
+     * execution and a "result" endpoint for DEFERRED requests that will deliver the result for a successful execution.
+     *
+     * @param host hostname used for this request
+     * @param path servlet path for this request
+     * @param delivery ResultDelivery mode for this request
+     * @return a handle (URL) that allows a client to access further information for this request
+     */
+    protected String getHandleUrl(String host, String path, ResultDelivery delivery) {
+        return "http://" + host + path + handlePath(delivery);
+    }
+
     private void handleRequest(RequestParameters param, IServletResponse response) throws IOException {
         LOGGER.info(param.toString());
         long elapsedStart = System.nanoTime();
@@ -402,7 +402,8 @@ public class QueryServiceServlet extends AbstractQueryApiServlet {
 
         ResultDelivery delivery = parseResultDelivery(param.mode);
 
-        SessionConfig sessionConfig = createSessionConfig(param, resultWriter);
+        String handleUrl = getHandleUrl(param.host, param.path, delivery);
+        SessionConfig sessionConfig = createSessionConfig(param, handleUrl, resultWriter);
         HttpUtil.setContentType(response, HttpUtil.ContentType.APPLICATION_JSON, HttpUtil.Encoding.UTF8);
 
         HttpResponseStatus status = HttpResponseStatus.OK;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/QueryStatusApiServlet.java
----------------------------------------------------------------------
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 9aa74c5..061ccc3 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
@@ -18,24 +18,23 @@
  */
 package org.apache.asterix.api.http.server;
 
+import static org.apache.asterix.api.http.server.AbstractQueryApiServlet.ResultStatus.FAILED;
+
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.util.List;
 import java.util.concurrent.ConcurrentMap;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
+import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.hyracks.api.dataset.DatasetJobRecord;
 import org.apache.hyracks.api.dataset.IHyracksDataset;
-import org.apache.hyracks.api.dataset.ResultSetId;
-import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.http.api.IServletRequest;
 import org.apache.hyracks.http.api.IServletResponse;
 import org.apache.hyracks.http.server.utils.HttpUtil;
 
-import com.fasterxml.jackson.databind.JsonNode;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
 import io.netty.handler.codec.http.HttpResponseStatus;
 
 public class QueryStatusApiServlet extends AbstractQueryApiServlet {
@@ -47,27 +46,26 @@ public class QueryStatusApiServlet extends AbstractQueryApiServlet {
 
     @Override
     protected void get(IServletRequest request, IServletResponse response) throws Exception {
-        String strHandle = request.getParameter("handle");
-        ObjectMapper om = new ObjectMapper();
-        JsonNode handle = parseHandle(om, strHandle, LOGGER);
+        final String strHandle = localPath(request);
+        final ResultHandle handle = ResultHandle.parse(strHandle);
         if (handle == null) {
             response.setStatus(HttpResponseStatus.BAD_REQUEST);
             return;
         }
-        JobId jobId = new JobId(handle.get(0).asLong());
-        ResultSetId rsId = new ResultSetId(handle.get(1).asLong());
 
         IHyracksDataset hds = getHyracksDataset();
-        ResultReader resultReader = new ResultReader(hds, jobId, rsId);
-
-        ResultStatus resultStatus = resultStatus(resultReader.getStatus());
+        ResultReader resultReader = new ResultReader(hds, handle.getJobId(), handle.getResultSetId());
 
-        if (resultStatus == null) {
+        final DatasetJobRecord.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);
+
         final StringWriter stringWriter = new StringWriter();
         final PrintWriter resultWriter = new PrintWriter(stringWriter);
 
@@ -75,12 +73,14 @@ public class QueryStatusApiServlet extends AbstractQueryApiServlet {
         HttpResponseStatus httpStatus = HttpResponseStatus.OK;
 
         resultWriter.print("{\n");
-        printStatus(resultWriter, resultStatus);
+        printStatus(resultWriter, resultStatus, (ex != null) || ResultStatus.SUCCESS == resultStatus);
 
         if (ResultStatus.SUCCESS == resultStatus) {
             String servletPath = servletPath(request).replace("status", "result");
-            String resHandle = "http://" + host(request) + servletPath + localPath(request);
-            printHandle(resultWriter, resHandle);
+            String resHandle = "http://" + host(request) + servletPath + strHandle;
+            printHandle(resultWriter, resHandle, false);
+        } else if (ex != null) {
+            printError(resultWriter, ex, false);
         }
 
         resultWriter.print("}\n");
@@ -95,19 +95,29 @@ public class QueryStatusApiServlet extends AbstractQueryApiServlet {
     }
 
     ResultStatus resultStatus(DatasetJobRecord.Status status) {
-        if (status == null) {
-            return null;
-        }
-        switch (status) {
+        switch (status.getState()) {
             case IDLE:
             case RUNNING:
                 return ResultStatus.RUNNING;
             case SUCCESS:
                 return ResultStatus.SUCCESS;
             case FAILED:
-                return ResultStatus.FAILED;
+                return FAILED;
             default:
                 return ResultStatus.FATAL;
         }
     }
+
+    Exception extractException(DatasetJobRecord.Status status) {
+        switch (status.getState()) {
+            case FAILED:
+                List<Exception> exceptions = status.getExceptions();
+                if (exceptions != null && !exceptions.isEmpty()) {
+                    return exceptions.get(0);
+                }
+                return null;
+            default:
+                return null;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
index d7edb23..7a587ef 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/http/server/RestApiServlet.java
@@ -114,8 +114,8 @@ public abstract class RestApiServlet extends AbstractServlet {
         }
 
         SessionConfig.ResultDecorator handlePrefix =
-                (AlgebricksAppendable app) -> app.append("{ \"").append("handle").append("\": ");
-        SessionConfig.ResultDecorator handlePostfix = (AlgebricksAppendable app) -> app.append(" }");
+                (AlgebricksAppendable app) -> app.append("{ \"").append("handle").append("\": \"");
+        SessionConfig.ResultDecorator handlePostfix = (AlgebricksAppendable app) -> app.append("\" }");
         SessionConfig sessionConfig =
                 new SessionConfig(response.writer(), format, null, null, handlePrefix, handlePostfix);
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/result/ResultHandle.java
----------------------------------------------------------------------
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 05eb967..7809a84 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
@@ -24,16 +24,51 @@ import org.apache.hyracks.api.dataset.ResultSetId;
 import org.apache.hyracks.api.job.JobId;
 
 public class ResultHandle {
-    private long jobId;
-    private long resultSetId;
+    private final JobId jobId;
+    private final ResultSetId resultSetId;
 
     public ResultHandle(JobId jobId, ResultSetId resultSetId) {
-        this.jobId = jobId.getId();
-        this.resultSetId = resultSetId.getId();
+        this.jobId = jobId;
+        this.resultSetId = resultSetId;
+    }
+
+    public ResultHandle(long jobId, long resultSetId) {
+        this(new JobId(jobId), new ResultSetId(resultSetId));
+    }
+
+    public static ResultHandle parse(String str) {
+        int dash = str.indexOf('-');
+        if (dash < 1) {
+            return null;
+        }
+        int start = 0;
+        while (str.charAt(start) == '/') {
+            ++start;
+        }
+        String jobIdStr = str.substring(start, dash);
+        String resIdStr = str.substring(dash + 1);
+        try {
+            return new ResultHandle(Long.parseLong(jobIdStr), Long.parseLong(resIdStr));
+        } catch (NumberFormatException e) {
+            return null;
+        }
+    }
+
+    public JobId getJobId() {
+        return jobId;
+    }
+
+    public ResultSetId getResultSetId() {
+        return resultSetId;
+    }
+
+    @Override
+    public String toString() {
+        return Long.toString(jobId.getId()) + "-" + Long.toString(resultSetId.getId());
+
     }
 
     public AlgebricksAppendable append(AlgebricksAppendable app) throws AlgebricksException {
-        return app.append("[").append(String.valueOf(jobId)).append(", ").append(String.valueOf(resultSetId))
-                .append("]");
+        return app.append(toString());
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
----------------------------------------------------------------------
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 26a6ebd..03bef13 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
@@ -164,6 +164,7 @@ import org.apache.asterix.utils.DataverseUtil;
 import org.apache.asterix.utils.FeedOperations;
 import org.apache.asterix.utils.FlushDatasetUtil;
 import org.apache.commons.lang3.StringUtils;
+import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableBoolean;
 import org.apache.commons.lang3.mutable.MutableObject;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
@@ -2405,10 +2406,10 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         switch (resultDelivery) {
             case ASYNC:
                 MutableBoolean printed = new MutableBoolean(false);
+                Mutable<JobId> jobId = new MutableObject<>(JobId.INVALID);
                 executorService.submit(() -> {
-                    JobId jobId = null;
                     try {
-                        jobId = createAndRunJob(hcc, compiler, locker, resultDelivery, id -> {
+                        createAndRunJob(hcc, jobId, compiler, locker, resultDelivery, id -> {
                             final ResultHandle handle = new ResultHandle(id, resultSetId);
                             ResultUtil.printResultHandle(handle, sessionConfig);
                             synchronized (printed) {
@@ -2417,8 +2418,11 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                             }
                         }, clientContextId, ctx);
                     } catch (Exception e) {
-                        GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE,
-                                resultDelivery.name() + " job " + "with id " + jobId + " failed", e);
+                        synchronized (jobId) {
+                            GlobalConfig.ASTERIX_LOGGER.log(Level.SEVERE,
+                                    resultDelivery.name() + " job " + "with id " + jobId.getValue() + " " + "failed",
+                                    e);
+                        }
                     }
                 });
                 synchronized (printed) {
@@ -2428,14 +2432,14 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 }
                 break;
             case IMMEDIATE:
-                createAndRunJob(hcc, compiler, locker, resultDelivery, id -> {
+                createAndRunJob(hcc, null, compiler, locker, resultDelivery, id -> {
                     final ResultReader resultReader = new ResultReader(hdc, id, resultSetId);
                     ResultUtil.printResults(resultReader, sessionConfig, stats,
                             metadataProvider.findOutputRecordType());
                 }, clientContextId, ctx);
                 break;
             case DEFERRED:
-                createAndRunJob(hcc, compiler, locker, resultDelivery, id -> {
+                createAndRunJob(hcc, null, compiler, locker, resultDelivery, id -> {
                     ResultUtil.printResultHandle(new ResultHandle(id, resultSetId), sessionConfig);
                 }, clientContextId, ctx);
                 break;
@@ -2444,20 +2448,24 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         }
     }
 
-    private static JobId createAndRunJob(IHyracksClientConnection hcc, IStatementCompiler compiler,
+    private static void createAndRunJob(IHyracksClientConnection hcc, Mutable<JobId> jId, IStatementCompiler compiler,
             IMetadataLocker locker, ResultDelivery resultDelivery, IResultPrinter printer, String clientContextId,
             IStatementExecutorContext ctx) throws Exception {
         locker.lock();
         try {
             final JobSpecification jobSpec = compiler.compile();
             if (jobSpec == null) {
-                return JobId.INVALID;
+                return;
             }
             final JobId jobId = JobUtils.runJob(hcc, jobSpec, false);
-
             if (ctx != null && clientContextId != null) {
                 ctx.put(clientContextId, jobId); // Adds the running job into the context.
             }
+            if (jId != null) {
+                synchronized (jId) {
+                    jId.setValue(jobId);
+                }
+            }
             if (ResultDelivery.ASYNC == resultDelivery) {
                 printer.print(jobId);
                 hcc.waitForCompletion(jobId);
@@ -2465,7 +2473,6 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 hcc.waitForCompletion(jobId);
                 printer.print(jobId);
             }
-            return jobId;
         } finally {
             // No matter the job succeeds or fails, removes it into the context.
             if (ctx != null && clientContextId != null) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
index 97101ba..5ab9c1f 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/CancellationTestExecutor.java
@@ -46,12 +46,11 @@ public class CancellationTestExecutor extends TestExecutor {
             List<TestCase.CompilationUnit.Parameter> params, boolean jsonEncoded, boolean cancellable)
             throws Exception {
         String clientContextId = UUID.randomUUID().toString();
-        if (cancellable) {
-            setParam(params, "client_context_id", clientContextId);
-        }
+        final List<TestCase.CompilationUnit.Parameter> newParams =
+                cancellable ? upsertParam(params, "client_context_id", clientContextId) : params;
         Callable<InputStream> query = () -> {
             try {
-                return CancellationTestExecutor.super.executeQueryService(str, fmt, uri, params, jsonEncoded, true);
+                return CancellationTestExecutor.super.executeQueryService(str, fmt, uri, newParams, jsonEncoded, true);
             } catch (Exception e) {
                 e.printStackTrace();
                 throw e;
@@ -61,7 +60,7 @@ public class CancellationTestExecutor extends TestExecutor {
         if (cancellable) {
             Thread.sleep(20);
             // Cancels the query request while the query is executing.
-            int rc = cancelQuery(getEndpoint(Servlets.RUNNING_REQUESTS), params);
+            int rc = cancelQuery(getEndpoint(Servlets.RUNNING_REQUESTS), newParams);
             Assert.assertTrue(rc == 200 || rc == 404);
         }
         InputStream inputStream = future.get();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
index 53a0f6c..1d50c26 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/ResultExtractor.java
@@ -107,7 +107,7 @@ public class ResultExtractor {
                     }
                     break;
                 default:
-                    throw new AsterixException(field + " unanticipated field");
+                    throw new AsterixException("Unanticipated field \"" + field + "\"");
             }
         }
 
@@ -119,9 +119,16 @@ public class ResultExtractor {
         ObjectMapper om = new ObjectMapper();
         String result = IOUtils.toString(resultStream, utf8);
         ObjectNode resultJson = om.readValue(result, ObjectNode.class);
-        JsonNode handle = resultJson.get("handle");
-        ObjectNode res = om.createObjectNode();
-        res.set("handle", handle);
-        return om.writeValueAsString(res);
+        final JsonNode handle = resultJson.get("handle");
+        if (handle != null) {
+            return handle.asText();
+        } else {
+            JsonNode errors = resultJson.get("errors");
+            if (errors != null) {
+                JsonNode msg = errors.get(0).get("msg");
+                throw new AsterixException(msg.asText());
+            }
+        }
+        return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
index 608547c..9a3de14 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestExecutor.java
@@ -76,6 +76,7 @@ import org.apache.http.impl.client.StandardHttpRequestRetryHandler;
 import org.apache.http.util.EntityUtils;
 import org.apache.hyracks.util.StorageUtil;
 import org.junit.Assert;
+
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.fasterxml.jackson.databind.DeserializationFeature;
 import com.fasterxml.jackson.databind.JsonMappingException;
@@ -461,43 +462,44 @@ public class TestExecutor {
 
     protected InputStream executeQueryService(String str, OutputFormat fmt, URI uri,
             List<CompilationUnit.Parameter> params, boolean jsonEncoded, boolean cancellable) throws Exception {
-        setParam(params, "format", fmt.mimeType());
-        HttpUriRequest method = jsonEncoded ? constructPostMethodJson(str, uri, "statement", params)
-                : constructPostMethodUrl(str, uri, "statement", params);
+        final List<CompilationUnit.Parameter> newParams = upsertParam(params, "format", fmt.mimeType());
+        HttpUriRequest method = jsonEncoded ? constructPostMethodJson(str, uri, "statement", newParams)
+                : constructPostMethodUrl(str, uri, "statement", newParams);
         // Set accepted output response type
         method.setHeader("Accept", OutputFormat.CLEAN_JSON.mimeType());
         HttpResponse response = executeHttpRequest(method);
         return response.getEntity().getContent();
     }
 
-    protected void setParam(List<CompilationUnit.Parameter> params, String name, String value) {
+    protected List<CompilationUnit.Parameter> upsertParam(List<CompilationUnit.Parameter> params, String name,
+            String value) {
+        boolean replaced = false;
+        List<CompilationUnit.Parameter> result = new ArrayList<>();
         for (CompilationUnit.Parameter param : params) {
+            CompilationUnit.Parameter newParam = new CompilationUnit.Parameter();
+            newParam.setName(param.getName());
             if (name.equals(param.getName())) {
-                param.setValue(value);
-                return;
+                newParam.setValue(value);
+                replaced = true;
+            } else {
+                newParam.setValue(param.getValue());
             }
+            result.add(newParam);
         }
-        CompilationUnit.Parameter formatParam = new CompilationUnit.Parameter();
-        formatParam.setName(name);
-        formatParam.setValue(value);
-        params.add(formatParam);
-    }
-
-    private List<CompilationUnit.Parameter> injectStatement(String statement, String stmtParamName,
-            List<CompilationUnit.Parameter> otherParams) {
-        CompilationUnit.Parameter stmtParam = new CompilationUnit.Parameter();
-        stmtParam.setName(stmtParamName);
-        stmtParam.setValue(statement);
-        List<CompilationUnit.Parameter> params = new ArrayList<>(otherParams);
-        params.add(stmtParam);
-        return params;
+        if (!replaced) {
+            CompilationUnit.Parameter newParam = new CompilationUnit.Parameter();
+            newParam.setName(name);
+            newParam.setValue(value);
+            result.add(newParam);
+        }
+        return result;
     }
 
     private HttpUriRequest constructHttpMethod(String statement, URI uri, String stmtParam, boolean postStmtAsParam,
             List<CompilationUnit.Parameter> otherParams) throws URISyntaxException {
         if (statement.length() + uri.toString().length() < MAX_URL_LENGTH) {
             // Use GET for small-ish queries
-            return constructGetMethod(uri, injectStatement(statement, stmtParam, otherParams));
+            return constructGetMethod(uri, upsertParam(otherParams, stmtParam, statement));
         } else {
             // Use POST for bigger ones to avoid 413 FULL_HEAD
             String stmtParamName = (postStmtAsParam ? stmtParam : null);
@@ -541,7 +543,7 @@ public class TestExecutor {
             List<CompilationUnit.Parameter> otherParams) {
         RequestBuilder builder = RequestBuilder.post(uri);
         if (stmtParam != null) {
-            for (CompilationUnit.Parameter param : injectStatement(statement, stmtParam, otherParams)) {
+            for (CompilationUnit.Parameter param : upsertParam(otherParams, stmtParam, statement)) {
                 builder.addParameter(param.getName(), param.getValue());
             }
             builder.addParameter(stmtParam, statement);
@@ -561,7 +563,7 @@ public class TestExecutor {
         RequestBuilder builder = RequestBuilder.post(uri);
         ObjectMapper om = new ObjectMapper();
         ObjectNode content = om.createObjectNode();
-        for (CompilationUnit.Parameter param : injectStatement(statement, stmtParam, otherParams)) {
+        for (CompilationUnit.Parameter param : upsertParam(otherParams, stmtParam, statement)) {
             content.put(param.getName(), param.getValue());
         }
         try {
@@ -609,10 +611,13 @@ public class TestExecutor {
 
         HttpResponse response = executeAndCheckHttpRequest(request);
         InputStream resultStream = response.getEntity().getContent();
-        String handle = IOUtils.toString(resultStream, "UTF-8");
+        String resultStr = IOUtils.toString(resultStream, "UTF-8");
+        ObjectNode resultJson = new ObjectMapper().readValue(resultStr, ObjectNode.class);
+        final JsonNode jsonHandle = resultJson.get("handle");
+        final String strHandle = jsonHandle.asText();
 
         if (handleVar != null) {
-            variableCtx.put(handleVar, handle);
+            variableCtx.put(handleVar, strHandle);
             return resultStream;
         }
         return null;
@@ -776,6 +781,7 @@ public class TestExecutor {
                 long startTime = System.currentTimeMillis();
                 long limitTime = startTime + TimeUnit.SECONDS.toMillis(timeoutSecs);
                 ctx.setType(ctx.getType().substring("poll".length()));
+                boolean expectedException = false;
                 Exception finalException;
                 LOGGER.fine("polling for up to " + timeoutSecs + " seconds w/ " + retryDelaySecs + " second(s) delay");
                 while (true) {
@@ -785,6 +791,11 @@ public class TestExecutor {
                         finalException = null;
                         break;
                     } catch (Exception e) {
+                        if (isExpected(e, cUnit)) {
+                            expectedException = true;
+                            finalException = e;
+                            break;
+                        }
                         if ((System.currentTimeMillis() > limitTime)) {
                             finalException = e;
                             break;
@@ -793,7 +804,9 @@ public class TestExecutor {
                         Thread.sleep(TimeUnit.SECONDS.toMillis(retryDelaySecs));
                     }
                 }
-                if (finalException != null) {
+                if (expectedException) {
+                    throw finalException;
+                } else if (finalException != null){
                     throw new Exception("Poll limit (" + timeoutSecs + "s) exceeded without obtaining expected result",
                             finalException);
                 }
@@ -838,8 +851,8 @@ public class TestExecutor {
                         resultStream = ResultExtractor.extract(resultStream);
                     } else {
                         String handleVar = getHandleVariable(statement);
-                        setParam(params, "mode", delivery);
-                        resultStream = executeQueryService(statement, fmt, uri, params, true);
+                        resultStream =
+                                executeQueryService(statement, fmt, uri, upsertParam(params, "mode", delivery), true);
                         String handle = ResultExtractor.extractHandle(resultStream);
                         Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), handleVar);
                         variableCtx.put(handleVar, handle);
@@ -946,19 +959,32 @@ public class TestExecutor {
                 break;
             case "get":
             case "post":
-                if (!"http".equals(ctx.extension())) {
-                    throw new IllegalArgumentException(
-                            "Unexpected format for method " + ctx.getType() + ": " + ctx.extension());
-                }
                 fmt = OutputFormat.forCompilationUnit(cUnit);
+                String handleVar = getHandleVariable(statement);
                 final String trimmedPathAndQuery = stripLineComments(stripJavaComments(statement)).trim();
                 final String variablesReplaced = replaceVarRef(trimmedPathAndQuery, variableCtx);
-                resultStream = executeHttp(ctx.getType(), variablesReplaced, fmt);
-                expectedResultFile = expectedResultFileCtxs.get(queryCount.intValue()).getFile();
-                actualResultFile = testCaseCtx.getActualResultFile(cUnit, expectedResultFile, new File(actualPath));
-                writeOutputToFile(actualResultFile, resultStream);
-                runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), expectedResultFile,
-                        actualResultFile);
+                if ("http".equals(ctx.extension())) {
+                    resultStream = executeHttp(ctx.getType(), variablesReplaced, fmt);
+                } else if ("uri".equals(ctx.extension())) {
+                    resultStream = executeURI(ctx.getType(), URI.create(variablesReplaced), fmt);
+                } else {
+                    throw new IllegalArgumentException(
+                            "Unexpected format for method " + ctx.getType() + ": " + ctx.extension());
+                }
+                if (handleVar != null) {
+                    String handle = ResultExtractor.extractHandle(resultStream);
+                    if (handle != null) {
+                        variableCtx.put(handleVar, handle);
+                    } else {
+                        throw new Exception("no handle for test " + testFile.toString());
+                    }
+                } else {
+                    expectedResultFile = expectedResultFileCtxs.get(queryCount.intValue()).getFile();
+                    actualResultFile = testCaseCtx.getActualResultFile(cUnit, expectedResultFile, new File(actualPath));
+                    writeOutputToFile(actualResultFile, resultStream);
+                    runScriptAndCompareWithResult(testFile, new PrintWriter(System.err), expectedResultFile,
+                            actualResultFile);
+                }
                 queryCount.increment();
                 break;
             case "server": // (start <test server name> <port>
@@ -1052,6 +1078,16 @@ public class TestExecutor {
         }
     }
 
+    protected static boolean isExpected(Exception e, CompilationUnit cUnit) {
+        final List<String> expErrors = cUnit.getExpectedError();
+        for (String exp : expErrors) {
+            if (e.toString().contains(exp)) {
+                return true;
+            }
+        }
+        return false;
+    }
+
     protected int getTimeoutSecs(String statement) {
         final Matcher timeoutMatcher = POLL_TIMEOUT_PATTERN.matcher(statement);
         if (timeoutMatcher.find()) {
@@ -1087,6 +1123,10 @@ public class TestExecutor {
     protected InputStream executeHttp(String ctxType, String endpoint, OutputFormat fmt) throws Exception {
         String[] split = endpoint.split("\\?");
         URI uri = new URI("http", null, host, port, split[0], split.length > 1 ? split[1] : null, null);
+        return executeURI(ctxType, uri, fmt);
+    }
+
+    private InputStream executeURI(String ctxType, URI uri, OutputFormat fmt) throws Exception {
         switch (ctxType) {
             case "get":
                 return executeJSONGet(fmt, uri);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_1/query_result_1.1.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_1/query_result_1.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_1/query_result_1.1.get.http
index d48dbe5..c676d2d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_1/query_result_1.1.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_1/query_result_1.1.get.http
@@ -22,4 +22,4 @@
  * Expected Result : Negative
  * Date            : 25th February 2017
  */
-/query/result?handle={"handle":[18,0]}
+/query/service/result/18-0

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_2/query_result_2.1.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_2/query_result_2.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_2/query_result_2.1.get.http
index 07b7556..effb6a5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_2/query_result_2.1.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_2/query_result_2.1.get.http
@@ -22,4 +22,4 @@
  * Expected Result : Negative
  * Date            : 25th February 2017
  */
-/query/result?handle={"handle":[18,0]
+/query/service/result/18_0

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_3/query_result_3.1.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_3/query_result_3.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_3/query_result_3.1.get.http
index c39b87e..33cebe9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_3/query_result_3.1.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_3/query_result_3.1.get.http
@@ -22,4 +22,4 @@
  * Expected Result : Negative
  * Date            : 25th February 2017
  */
-/query/result?handle
+/query/service/result/

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_4/query_result_4.1.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_4/query_result_4.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_4/query_result_4.1.get.http
index 88c4814..a949a28 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_4/query_result_4.1.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_result_4/query_result_4.1.get.http
@@ -22,4 +22,4 @@
  * Expected Result : Negative
  * Date            : 25th February 2017
  */
-/query/result?handl={"handle":[18,0]}
+/query/service/result

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_1/query_status_1.1.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_1/query_status_1.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_1/query_status_1.1.get.http
index c18a55b..66428a8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_1/query_status_1.1.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_1/query_status_1.1.get.http
@@ -22,4 +22,4 @@
  * Expected Result : Negative
  * Date            : 7th September 2016
  */
-/query/status?handle={"handle":[18,0]}
+/query/service/status/18-0

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_2/query_status_2.1.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_2/query_status_2.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_2/query_status_2.1.get.http
index d7ece4c..60634d3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_2/query_status_2.1.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_2/query_status_2.1.get.http
@@ -22,4 +22,4 @@
  * Expected Result : Negative
  * Date            : 7th September 2016
  */
-/query/status?handle={"handle":[18,0]
+/query/service/status/18_0

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_3/query_status_3.1.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_3/query_status_3.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_3/query_status_3.1.get.http
index a7ff551..d7d7a0d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_3/query_status_3.1.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_3/query_status_3.1.get.http
@@ -22,4 +22,4 @@
  * Expected Result : Negative
  * Date            : 7th September 2016
  */
-/query/status?handle
+/query/service/status/

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_4/query_status_4.1.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_4/query_status_4.1.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_4/query_status_4.1.get.http
index 5831686..23930dd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_4/query_status_4.1.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/query_status_4/query_status_4.1.get.http
@@ -22,4 +22,4 @@
  * Expected Result : Negative
  * Date            : 7th September 2016
  */
-/query/status?handl={"handle":[18,0]}
+/query/service/status

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
index a88991c..8417a7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.get.http
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-/query/result?handle=$handle
+/query/service/result/$handle

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
index 5d59ca3..bcc0edc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.7.pollget.http
@@ -19,4 +19,4 @@
 
 //polltimeoutsecs=10
 
-/query/status?handle=$handle
+/query/service/status/$handle

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
index a88991c..8417a7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.8.get.http
@@ -17,4 +17,4 @@
  * under the License.
  */
 
-/query/result?handle=$handle
+/query/service/result/$handle

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.1.async.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.1.async.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.1.async.sqlpp
index 89ef35e..2f0feff 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.1.async.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.1.async.sqlpp
@@ -17,7 +17,7 @@
  * under the License.
  */
 
-#handlevariable=handle
+#handlevariable=status
 
 set `import-private-functions` `true`;
 select value inject_failure(sleep("result", 5000), true);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.2.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.2.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.2.pollget.http
deleted file mode 100644
index 916aadf..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.2.pollget.http
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#polltimeoutsecs=10
-
-/query/status?handle=$handle

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.2.pollget.uri
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.2.pollget.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.2.pollget.uri
new file mode 100644
index 0000000..e20319a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.2.pollget.uri
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+#polltimeoutsecs=30
+#handlevariable=result
+
+$status

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.3.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.3.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.3.get.http
deleted file mode 100644
index a88991c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-failed/async-failed.3.get.http
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/query/result?handle=$handle

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.1.async.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.1.async.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.1.async.sqlpp
new file mode 100644
index 0000000..c9a2958
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.1.async.sqlpp
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+#handlevariable=status
+
+select i, i * i as i2 from range(1, 10) i;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.2.pollget.uri
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.2.pollget.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.2.pollget.uri
new file mode 100644
index 0000000..bf3f04c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.2.pollget.uri
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+#polltimeoutsecs=20
+#handlevariable=result
+
+$status

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.3.get.uri
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.3.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.3.get.uri
new file mode 100644
index 0000000..b613531
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.3.get.uri
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+$result

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.4.get.uri
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.4.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.4.get.uri
new file mode 100644
index 0000000..b613531
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.4.get.uri
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+$result

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.5.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.5.query.sqlpp
new file mode 100644
index 0000000..e452678
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-repeated/async-repeated.5.query.sqlpp
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+select i, i * i as i2 from range(1, 10) i;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.1.async.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.1.async.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.1.async.sqlpp
index 866b388..5237950 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.1.async.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.1.async.sqlpp
@@ -17,6 +17,6 @@
  * under the License.
  */
 
-#handlevariable=handle
+#handlevariable=status
 
-select value sleep("result", 3000);
+select value sleep("result", 5000);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.2.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.2.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.2.pollget.http
deleted file mode 100644
index 916aadf..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.2.pollget.http
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#polltimeoutsecs=10
-
-/query/status?handle=$handle

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.2.pollget.uri
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.2.pollget.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.2.pollget.uri
new file mode 100644
index 0000000..c83909b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.2.pollget.uri
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+#polltimeoutsecs=10
+
+$status

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.3.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.3.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.3.pollget.http
deleted file mode 100644
index 916aadf..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.3.pollget.http
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#polltimeoutsecs=10
-
-/query/status?handle=$handle

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.3.pollget.uri
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.3.pollget.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.3.pollget.uri
new file mode 100644
index 0000000..12fcdfd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.3.pollget.uri
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+#polltimeoutsecs=10
+#handlevariable=result
+
+$status

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.4.get.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.4.get.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.4.get.http
deleted file mode 100644
index a88991c..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.4.get.http
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-/query/result?handle=$handle

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.4.get.uri
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.4.get.uri b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.4.get.uri
new file mode 100644
index 0000000..b613531
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async-running/async-running.4.get.uri
@@ -0,0 +1,20 @@
+/*
+ * 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.
+ */
+
+$result

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.1.async.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.1.async.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.1.async.sqlpp
index a44b911..c9a2958 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.1.async.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.1.async.sqlpp
@@ -17,6 +17,6 @@
  * under the License.
  */
 
-#handlevariable=handle
+#handlevariable=status
 
 select i, i * i as i2 from range(1, 10) i;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/d8a6c71d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.2.pollget.http
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.2.pollget.http b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.2.pollget.http
deleted file mode 100644
index 916aadf..0000000
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/async-deferred/async/async.2.pollget.http
+++ /dev/null
@@ -1,22 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-#polltimeoutsecs=10
-
-/query/status?handle=$handle