You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by sj...@apache.org on 2018/06/15 15:24:05 UTC

[2/3] asterixdb git commit: [ASTERIXDB-2401][SQLPP] Support parameterized queries

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/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 4697a2a..2281238 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
@@ -30,6 +30,7 @@ import java.io.OutputStream;
 import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.math.BigDecimal;
 import java.net.Inet4Address;
 import java.net.InetSocketAddress;
 import java.net.Socket;
@@ -69,6 +70,7 @@ import org.apache.asterix.testframework.context.TestCaseContext;
 import org.apache.asterix.testframework.context.TestCaseContext.OutputFormat;
 import org.apache.asterix.testframework.context.TestFileContext;
 import org.apache.asterix.testframework.xml.ComparisonEnum;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
 import org.apache.asterix.testframework.xml.TestCase.CompilationUnit;
 import org.apache.asterix.testframework.xml.TestCase.CompilationUnit.Parameter;
 import org.apache.asterix.testframework.xml.TestGroup;
@@ -90,10 +92,13 @@ import org.apache.http.impl.client.HttpClients;
 import org.apache.http.impl.client.StandardHttpRequestRetryHandler;
 import org.apache.http.protocol.HttpContext;
 import org.apache.http.util.EntityUtils;
+import org.apache.hyracks.http.server.utils.HttpUtil;
 import org.apache.hyracks.util.StorageUtil;
 import org.apache.logging.log4j.Level;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
+
+import com.fasterxml.jackson.databind.util.RawValue;
 import org.junit.Assert;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
@@ -124,11 +129,13 @@ public class TestExecutor {
     private static final Pattern POLL_DELAY_PATTERN = Pattern.compile("polldelaysecs=(\\d+)(\\D|$)", Pattern.MULTILINE);
     private static final Pattern HANDLE_VARIABLE_PATTERN = Pattern.compile("handlevariable=(\\w+)");
     private static final Pattern VARIABLE_REF_PATTERN = Pattern.compile("\\$(\\w+)");
-    private static final Pattern HTTP_PARAM_PATTERN = Pattern.compile("param (\\w+)=(.*)", Pattern.MULTILINE);
+    private static final Pattern HTTP_PARAM_PATTERN =
+            Pattern.compile("param (?<name>[\\w$]+)(?::(?<type>\\w+))?=(?<value>.*)", Pattern.MULTILINE);
     private static final Pattern HTTP_BODY_PATTERN = Pattern.compile("body=(.*)", Pattern.MULTILINE);
     private static final Pattern HTTP_STATUSCODE_PATTERN = Pattern.compile("statuscode (.*)", Pattern.MULTILINE);
     private static final Pattern MAX_RESULT_READS_PATTERN =
             Pattern.compile("maxresultreads=(\\d+)(\\D|$)", Pattern.MULTILINE);
+    private static final Pattern HTTP_REQUEST_TYPE = Pattern.compile("requesttype=(.*)", Pattern.MULTILINE);
     public static final int TRUNCATE_THRESHOLD = 16384;
     public static final Set<String> NON_CANCELLABLE =
             Collections.unmodifiableSet(new HashSet<>(Arrays.asList("store", "validate")));
@@ -579,15 +586,15 @@ public class TestExecutor {
 
     public InputStream executeQueryService(String str, OutputFormat fmt, URI uri, List<Parameter> params,
             boolean jsonEncoded, Predicate<Integer> responseCodeValidator, boolean cancellable) throws Exception {
-        List<Parameter> newParams = upsertParam(params, "format", fmt.mimeType());
+        List<Parameter> newParams = upsertParam(params, "format", ParameterTypeEnum.STRING, fmt.mimeType());
         final Optional<String> maxReadsOptional = extractMaxResultReads(str);
         if (maxReadsOptional.isPresent()) {
             newParams = upsertParam(newParams, QueryServiceServlet.Parameter.MAX_RESULT_READS.str(),
-                    maxReadsOptional.get());
+                    ParameterTypeEnum.STRING, maxReadsOptional.get());
         }
         final List<Parameter> additionalParams = extractParameters(str);
         for (Parameter param : additionalParams) {
-            newParams = upsertParam(newParams, param.getName(), param.getValue());
+            newParams = upsertParam(newParams, param.getName(), param.getType(), param.getValue());
         }
         HttpUriRequest method = jsonEncoded ? constructPostMethodJson(str, uri, "statement", newParams)
                 : constructPostMethodUrl(str, uri, "statement", newParams);
@@ -600,16 +607,18 @@ public class TestExecutor {
         return response.getEntity().getContent();
     }
 
-    protected List<Parameter> upsertParam(List<Parameter> params, String name, String value) {
+    protected List<Parameter> upsertParam(List<Parameter> params, String name, ParameterTypeEnum type, String value) {
         boolean replaced = false;
         List<Parameter> result = new ArrayList<>();
         for (Parameter param : params) {
             Parameter newParam = new Parameter();
             newParam.setName(param.getName());
             if (name.equals(param.getName())) {
+                newParam.setType(type);
                 newParam.setValue(value);
                 replaced = true;
             } else {
+                newParam.setType(param.getType());
                 newParam.setValue(param.getValue());
             }
             result.add(newParam);
@@ -617,6 +626,7 @@ public class TestExecutor {
         if (!replaced) {
             Parameter newParam = new Parameter();
             newParam.setName(name);
+            newParam.setType(type);
             newParam.setValue(value);
             result.add(newParam);
         }
@@ -677,7 +687,7 @@ public class TestExecutor {
             List<Parameter> otherParams) {
         RequestBuilder builder = RequestBuilder.post(uri);
         if (stmtParam != null) {
-            for (Parameter param : upsertParam(otherParams, stmtParam, statement)) {
+            for (Parameter param : upsertParam(otherParams, stmtParam, ParameterTypeEnum.STRING, statement)) {
                 builder.addParameter(param.getName(), param.getValue());
             }
             builder.addParameter(stmtParam, statement);
@@ -697,8 +707,23 @@ public class TestExecutor {
         RequestBuilder builder = RequestBuilder.post(uri);
         ObjectMapper om = new ObjectMapper();
         ObjectNode content = om.createObjectNode();
-        for (Parameter param : upsertParam(otherParams, stmtParam, statement)) {
-            content.put(param.getName(), param.getValue());
+        for (Parameter param : upsertParam(otherParams, stmtParam, ParameterTypeEnum.STRING, statement)) {
+            String paramName = param.getName();
+            ParameterTypeEnum paramType = param.getType();
+            if (paramType == null) {
+                paramType = ParameterTypeEnum.STRING;
+            }
+            String paramValue = param.getValue();
+            switch (paramType) {
+                case STRING:
+                    content.put(paramName, paramValue);
+                    break;
+                case JSON:
+                    content.putRawValue(paramName, new RawValue(paramValue));
+                    break;
+                default:
+                    throw new IllegalStateException(paramType.toString());
+            }
         }
         try {
             builder.setEntity(new StringEntity(om.writeValueAsString(content), ContentType.APPLICATION_JSON));
@@ -1178,14 +1203,17 @@ public class TestExecutor {
         }
         URI uri = testFile.getName().endsWith("aql") ? getEndpoint(Servlets.QUERY_AQL)
                 : getEndpoint(Servlets.QUERY_SERVICE);
+        boolean isJsonEncoded = isJsonEncoded(extractHttpRequestType(statement));
         InputStream resultStream;
         if (DELIVERY_IMMEDIATE.equals(delivery)) {
-            resultStream = executeQueryService(statement, fmt, uri, params, true, null, isCancellable(reqType));
+            resultStream =
+                    executeQueryService(statement, fmt, uri, params, isJsonEncoded, null, isCancellable(reqType));
             resultStream = METRICS_QUERY_TYPE.equals(reqType) ? ResultExtractor.extractMetrics(resultStream)
                     : ResultExtractor.extract(resultStream);
         } else {
             String handleVar = getHandleVariable(statement);
-            resultStream = executeQueryService(statement, fmt, uri, upsertParam(params, "mode", delivery), true);
+            resultStream = executeQueryService(statement, fmt, uri,
+                    upsertParam(params, "mode", ParameterTypeEnum.STRING, delivery), isJsonEncoded);
             String handle = ResultExtractor.extractHandle(resultStream);
             Assert.assertNotNull("no handle for " + reqType + " test " + testFile.toString(), handleVar);
             variableCtx.put(handleVar, toQueryServiceHandle(handle));
@@ -1445,18 +1473,49 @@ public class TestExecutor {
         return Optional.empty();
     }
 
-    protected static List<Parameter> extractParameters(String statement) {
+    public static List<Parameter> extractParameters(String statement) {
         List<Parameter> params = new ArrayList<>();
         final Matcher m = HTTP_PARAM_PATTERN.matcher(statement);
         while (m.find()) {
             final Parameter param = new Parameter();
-            param.setName(m.group(1));
-            param.setValue(m.group(2));
+            String name = m.group("name");
+            param.setName(name);
+            String value = m.group("value");
+            param.setValue(value);
+            String type = m.group("type");
+            if (type != null) {
+                try {
+                    param.setType(ParameterTypeEnum.fromValue(type.toLowerCase()));
+                } catch (IllegalArgumentException e) {
+                    throw new IllegalArgumentException(
+                            String.format("Invalid type '%s' specified for parameter '%s'", type, name));
+                }
+            }
             params.add(param);
         }
         return params;
     }
 
+    private static String extractHttpRequestType(String statement) {
+        Matcher m = HTTP_REQUEST_TYPE.matcher(statement);
+        return m.find() ? m.group(1) : null;
+    }
+
+    private static boolean isJsonEncoded(String httpRequestType) throws Exception {
+        if (httpRequestType == null || httpRequestType.isEmpty()) {
+            return true;
+        }
+        switch (httpRequestType.trim()) {
+            case HttpUtil.ContentType.JSON:
+            case HttpUtil.ContentType.APPLICATION_JSON:
+                return true;
+            case HttpUtil.ContentType.APPLICATION_X_WWW_FORM_URLENCODED:
+                return false;
+            default:
+                throw new Exception("Invalid value for http request type: " + httpRequestType);
+        }
+    }
+
     protected static Predicate<Integer> extractStatusCodePredicate(String statement) {
         List<Integer> codes = new ArrayList<>();
         final Matcher m = HTTP_STATUSCODE_PATTERN.matcher(statement);
@@ -1511,6 +1570,7 @@ public class TestExecutor {
         List<Parameter> params = new ArrayList<>();
         Parameter node = new Parameter();
         node.setName("node");
+        node.setType(ParameterTypeEnum.STRING);
         node.setValue(nodeId);
         params.add(node);
         InputStream executeJSON = executeJSON(fmt, "POST", URI.create("http://localhost:16001" + endpoint), params);
@@ -1798,6 +1858,7 @@ public class TestExecutor {
         Stream.of("partition", "host", "port").forEach(arg -> {
             Parameter p = new Parameter();
             p.setName(arg);
+            p.setType(ParameterTypeEnum.STRING);
             parameters.add(p);
         });
         parameters.get(0).setValue(partition);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
index 66a34ff..a7a7fce 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/common/TestHelper.java
@@ -23,15 +23,29 @@ import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Collections;
 import java.util.Enumeration;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
 import java.util.zip.ZipEntry;
 import java.util.zip.ZipFile;
 
+import org.apache.asterix.api.http.server.QueryServiceServlet;
+import org.apache.asterix.app.translator.RequestParameters;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.testframework.xml.ParameterTypeEnum;
+import org.apache.asterix.testframework.xml.TestCase;
 import org.apache.commons.compress.utils.IOUtils;
 import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.util.file.FileUtil;
 
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.MapperFeature;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.fasterxml.jackson.databind.SerializationFeature;
+import com.fasterxml.jackson.databind.node.TextNode;
+
 public final class TestHelper {
 
     private static final String TEST_DIR_BASE_PATH = System.getProperty("user.dir") + File.separator + "target";
@@ -81,4 +95,53 @@ public final class TestHelper {
             }
         }
     }
+
+    public static Map<String, IAObject> readStatementParameters(String statement) throws IOException {
+        List<TestCase.CompilationUnit.Parameter> parameterList = TestExecutor.extractParameters(statement);
+        if (parameterList.isEmpty()) {
+            return Collections.emptyMap();
+        }
+        Map<String, JsonNode> stmtParams = new HashMap<>();
+        ObjectMapper om = createObjectMapper();
+        for (TestCase.CompilationUnit.Parameter param : parameterList) {
+            String paramName = param.getName();
+            JsonNode paramJsonValue;
+            ParameterTypeEnum paramType = param.getType();
+            if (paramType == null) {
+                paramType = ParameterTypeEnum.STRING;
+            }
+            String paramValue = param.getValue();
+            switch (paramType) {
+                case STRING:
+                    paramJsonValue = TextNode.valueOf(paramValue);
+                    break;
+                case JSON:
+                    paramJsonValue = om.readTree(paramValue);
+                    break;
+                default:
+                    throw new IllegalArgumentException(String.valueOf(paramType));
+
+            }
+            String name = QueryServiceServlet.extractStatementParameterName(paramName);
+            if (name != null) {
+                stmtParams.put(name, paramJsonValue);
+            } else if (QueryServiceServlet.Parameter.ARGS.str().equals(paramName)) {
+                if (paramJsonValue.isArray()) {
+                    for (int i = 0, ln = paramJsonValue.size(); i < ln; i++) {
+                        stmtParams.put(String.valueOf(i + 1), paramJsonValue.get(i));
+                    }
+                }
+            }
+        }
+
+        return RequestParameters.deserializeParameterValues(RequestParameters.serializeParameterValues(stmtParams));
+    }
+
+    private static ObjectMapper createObjectMapper() {
+        ObjectMapper objectMapper = new ObjectMapper();
+        objectMapper.enable(SerializationFeature.INDENT_OUTPUT);
+        objectMapper.configure(MapperFeature.SORT_PROPERTIES_ALPHABETICALLY, true);
+        objectMapper.configure(SerializationFeature.ORDER_MAP_ENTRIES_BY_KEYS, true);
+        return objectMapper;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
index e7b6271..0751178 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/jsonplan/JsonLogicalPlanTest.java
@@ -24,9 +24,11 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
-import java.io.Reader;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Map;
 
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.api.java.AsterixJavaClient;
@@ -41,11 +43,13 @@ import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.test.base.AsterixTestHelper;
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.test.runtime.HDFSCluster;
 import org.apache.asterix.translator.IStatementExecutorFactory;
 import org.apache.asterix.translator.SessionConfig.PlanFormat;
+import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
@@ -173,31 +177,28 @@ public class JsonLogicalPlanTest {
             Assume.assumeTrue(!skipped);
 
             LOGGER.info("RUN TEST: \"" + queryFile.getPath() + "\"");
-            Reader query = new BufferedReader(new InputStreamReader(new FileInputStream(queryFile), "UTF-8"));
+            String query = FileUtils.readFileToString(queryFile, StandardCharsets.UTF_8);
+            Map<String, IAObject> queryParams = TestHelper.readStatementParameters(query);
 
             // Forces the creation of actualFile.
             actualFile.getParentFile().mkdirs();
 
-            PrintWriter plan = new PrintWriter(actualFile);
             ILangCompilationProvider provider =
                     queryFile.getName().endsWith("aql") ? aqlCompilationProvider : sqlppCompilationProvider;
             if (extensionLangCompilationProvider != null) {
                 provider = extensionLangCompilationProvider;
             }
             IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
-            AsterixJavaClient asterix =
-                    new AsterixJavaClient((ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
-                            query, plan, provider, statementExecutorFactory, storageComponentProvider);
-            try {
-                asterix.compile(true, false, !optimized, optimized, false, false, false, PlanFormat.JSON);
 
+            try (PrintWriter plan = new PrintWriter(actualFile)) {
+                AsterixJavaClient asterix = new AsterixJavaClient(
+                        (ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
+                        new StringReader(query), plan, provider, statementExecutorFactory, storageComponentProvider);
+                asterix.setStatementParameters(queryParams);
+                asterix.compile(true, false, !optimized, optimized, false, false, false, PlanFormat.JSON);
             } catch (AsterixException e) {
-                plan.close();
-                query.close();
                 throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
             }
-            plan.close();
-            query.close();
 
             BufferedReader readerActual =
                     new BufferedReader(new InputStreamReader(new FileInputStream(actualFile), "UTF-8"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index 3fd59a4..debb3f9 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -23,14 +23,16 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.InputStreamReader;
 import java.io.PrintWriter;
-import java.io.Reader;
+import java.io.StringReader;
+import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
+import java.util.Map;
 
 import org.apache.asterix.api.common.AsterixHyracksIntegrationUtil;
 import org.apache.asterix.api.java.AsterixJavaClient;
 import org.apache.asterix.app.translator.DefaultStatementExecutorFactory;
-import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.context.IStorageComponentProvider;
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
 import org.apache.asterix.compiler.provider.AqlCompilationProvider;
@@ -39,10 +41,12 @@ import org.apache.asterix.compiler.provider.SqlppCompilationProvider;
 import org.apache.asterix.external.util.ExternalDataConstants;
 import org.apache.asterix.external.util.IdentitiyResolverFactory;
 import org.apache.asterix.file.StorageComponentProvider;
+import org.apache.asterix.om.base.IAObject;
 import org.apache.asterix.test.base.AsterixTestHelper;
 import org.apache.asterix.test.common.TestHelper;
 import org.apache.asterix.test.runtime.HDFSCluster;
 import org.apache.asterix.translator.IStatementExecutorFactory;
+import org.apache.commons.io.FileUtils;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.logging.log4j.LogManager;
@@ -113,7 +117,9 @@ public class OptimizerTest {
 
     private static void suiteBuildPerFile(File file, Collection<Object[]> testArgs, String path) {
         if (file.isDirectory() && !file.getName().startsWith(".")) {
-            for (File innerfile : file.listFiles()) {
+            File[] files = file.listFiles();
+            Arrays.sort(files);
+            for (File innerfile : files) {
                 String subdir = innerfile.isDirectory() ? path + innerfile.getName() + SEPARATOR : path;
                 suiteBuildPerFile(innerfile, testArgs, subdir);
             }
@@ -170,32 +176,29 @@ public class OptimizerTest {
             Assume.assumeTrue(!skipped);
 
             LOGGER.info("RUN TEST: \"" + queryFile.getPath() + "\"");
-            Reader query = new BufferedReader(new InputStreamReader(new FileInputStream(queryFile), "UTF-8"));
+            String query = FileUtils.readFileToString(queryFile, StandardCharsets.UTF_8);
+            Map<String, IAObject> queryParams = TestHelper.readStatementParameters(query);
 
             LOGGER.info("ACTUAL RESULT FILE: " + actualFile.getAbsolutePath());
 
             // Forces the creation of actualFile.
             actualFile.getParentFile().mkdirs();
 
-            PrintWriter plan = new PrintWriter(actualFile);
             ILangCompilationProvider provider =
                     queryFile.getName().endsWith("aql") ? aqlCompilationProvider : sqlppCompilationProvider;
             if (extensionLangCompilationProvider != null) {
                 provider = extensionLangCompilationProvider;
             }
             IHyracksClientConnection hcc = integrationUtil.getHyracksClientConnection();
-            AsterixJavaClient asterix =
-                    new AsterixJavaClient((ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
-                            query, plan, provider, statementExecutorFactory, storageComponentProvider);
-            try {
+            try (PrintWriter plan = new PrintWriter(actualFile)) {
+                AsterixJavaClient asterix = new AsterixJavaClient(
+                        (ICcApplicationContext) integrationUtil.cc.getApplicationContext(), hcc,
+                        new StringReader(query), plan, provider, statementExecutorFactory, storageComponentProvider);
+                asterix.setStatementParameters(queryParams);
                 asterix.compile(true, false, false, true, true, false, false);
             } catch (AlgebricksException e) {
-                plan.close();
-                query.close();
                 throw new Exception("Compile ERROR for " + queryFile + ": " + e.getMessage(), e);
             }
-            plan.close();
-            query.close();
 
             BufferedReader readerExpected =
                     new BufferedReader(new InputStreamReader(new FileInputStream(expectedFile), "UTF-8"));

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
index 0d0c1d9..830307b 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/sqlpp/ParserTestExecutor.java
@@ -188,8 +188,8 @@ public class ParserTestExecutor extends TestExecutor {
         PA.invokeMethod(rewriter,
                 "setup(java.util.List, org.apache.asterix.lang.common.base.IReturningStatement, "
                         + "org.apache.asterix.metadata.declared.MetadataProvider, "
-                        + "org.apache.asterix.lang.common.rewrites.LangRewritingContext)",
-                declaredFunctions, topExpr, metadataProvider, context);
+                        + "org.apache.asterix.lang.common.rewrites.LangRewritingContext, " + "java.util.Collection)",
+                declaredFunctions, topExpr, metadataProvider, context, null);
         PA.invokeMethod(rewriter, "inlineColumnAlias()");
         PA.invokeMethod(rewriter, "generateColumnNames()");
         PA.invokeMethod(rewriter, "substituteGroupbyKeyExpression()");

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp
new file mode 100644
index 0000000..0a53f64
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-01.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test plan for a query with named parameters
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+
+// param $p_str:json="hello"
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/statement-params_statement-params-01.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType)
+primary key c_id;
+
+create index idx_s on TestOpen(c_s:string);
+
+select c_id
+from  TestOpen as t
+where t.c_s = $p_str
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp
new file mode 100644
index 0000000..71e9f3c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-02.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test plan for a query with named parameters
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+
+// param args:json=["hello"]
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/statement-params_statement-params-03.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType)
+primary key c_id;
+
+create index idx_s on TestOpen(c_s:string);
+
+select c_id
+from  TestOpen as t
+where t.c_s = $1
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp
new file mode 100644
index 0000000..9019f68
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/statement-params/statement-params-03.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test plan for a query with named parameters
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+
+// param args:json=["hello"]
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/statement-params_statement-params-03.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType)
+primary key c_id;
+
+create index idx_s on TestOpen(c_s:string);
+
+select c_id
+from  TestOpen as t
+where t.c_s = ?
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan
new file mode 100644
index 0000000..cd5c81e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-01.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan
new file mode 100644
index 0000000..cd5c81e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-02.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan
new file mode 100644
index 0000000..cd5c81e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/statement-params/statement-params-03.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- STREAM_SELECT  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              -- BTREE_SEARCH  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp b/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp
index bca9e57..ceddca3 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/parserts/queries_sqlpp/ANYInFieldAccessor.sqlpp
@@ -20,5 +20,5 @@
 select element {'name':user.name,'movie':mv.movie}
 from  User as user,
       Movie as mv
-where some i in user.interests satisfies (i.movie = mv.movie[?])
+where some i in user.interests satisfies (i.movie = mv.movie[0])
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
index 42e81f9..3e3575d 100644
--- a/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
+++ b/asterixdb/asterix-app/src/test/resources/parserts/results_parser_sqlpp/ANYInFieldAccessor.ast
@@ -50,7 +50,7 @@ Where
             Variable [ Name=$mv ]
             Field=movie
           ]
-          Index: ANY
+          Index:           LiteralExpr [LONG] [0]
         ]
       ]
     ]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp
index d5afe16..0be9caa 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_06/customer_q_06.3.query.sqlpp
@@ -24,5 +24,5 @@ select element {'customerid':c.name,'orderedlist':n}
 from  Customers as c
 with  rec as c.lastorder,
       m as [c.cid,rec.oid],
-      n as [m[?],m[1],m[4]]
+      n as [m[0],m[1],m[4]]
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp
index 3344c77..71a0714 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/customer_q_07/customer_q_07.3.query.sqlpp
@@ -24,5 +24,5 @@ select element {'customerid':c.name,'unorderedlist':n}
 from  Customers as c
 with  rec as c.lastorder,
       m as [c.cid,rec.oid],
-      n as {{m[?],m[1],m[4]}}
+      n as {{m[0],m[1],m[4]}}
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp
index 7b34440..fac0fa5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/join_q_03/join_q_03.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element {'cust_name':c.name,'order_total':o.total,'orderedlist':ol,'unorderedlist':ul,'ol_item1':ol[0],'ol_item2':ol[1],'ol_item5':ol[4],'ul_item1':ul[?]}
+select element {'cust_name':c.name,'order_total':o.total,'orderedlist':ol,'unorderedlist':ul,'ol_item1':ol[0],'ol_item2':ol[1],'ol_item5':ol[4],'ul_item1':ul[0]}
 from  Customers as c,
       Orders as o
 with  rec as c.lastorder,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp
index 7271316..83a01cb 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_05/order_q_05.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element {'orderid':o.oid,'ordertot':o.total,'emptyorderedlist':c1,'emptyunorderedlist':c2,'olist_item1':c1[0],'olist_item5':c1[4],'ulist_item1':c2[?]}
+select element {'orderid':o.oid,'ordertot':o.total,'emptyorderedlist':c1,'emptyunorderedlist':c2,'olist_item1':c1[0],'olist_item5':c1[4],'ulist_item1':c2[0]}
 from  Orders as o
 with  c1 as [],
       c2 as {{}}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp
index a8859c5..cc8bc88 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/custord/order_q_06/order_q_06.3.query.sqlpp
@@ -20,7 +20,7 @@
 use test;
 
 
-select element {'item1':c3[?]}
+select element {'item1':c3[0]}
 from  Orders as o
 with  c3 as {{o.heList,o.openlist}}
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp
index 7c8f2ea..ea7e438 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/employee/q_02/q_02.3.query.sqlpp
@@ -22,5 +22,5 @@ use test;
 
 select element m
 from  Emp as e
-with  m as [{'EmpName':e.name,'parent_interest_1':e.interests[?],'child1Name':e.children[?],'child2Name':e.children[1]}]
+with  m as [{'EmpName':e.name,'parent_interest_1':e.interests[0],'child1Name':e.children[0],'child2Name':e.children[1]}]
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp
index 208a440..ebccaf3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/list/any-collection-member_01/any-collection-member_01.3.query.sqlpp
@@ -20,4 +20,4 @@
 use test;
 
 
-{{1,1,1}}[?];
+{{1,1,1}}[0];

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.1.query.sqlpp
new file mode 100644
index 0000000..1763959
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.1.query.sqlpp
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named and positional statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[2.5, " world"]
+// param $p_int:json=2
+// param $p_str:json="hello"
+
+{
+  "t1": $p_int + ? + $1,
+  "t2": $p_str || ? || $2
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.2.query.sqlpp
new file mode 100644
index 0000000..c01c253
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/mixed_01/mixed_01.2.query.sqlpp
@@ -0,0 +1,33 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named and positional statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[" b", " c"]
+// param $p_str:json="a"
+
+{
+  "t1": $p_str || ? || $1 || ? || $2
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.1.query.sqlpp
new file mode 100644
index 0000000..68d6aad
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.1.query.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param $p_null:json=null
+// param $p_bool:json=true
+// param $p_int:json=42
+// param $p_dec:json=42.5
+// param $p_dbl:json=42.5e2
+// param $p_str:json="hello"
+// param $p_arr:json=["99",100,{"a":null},null,true]
+// param $p_obj:json={"a":[1,2,3]}
+
+{
+  "t1": {
+    "p_null": $p_null,
+    "p_bool": $p_bool,
+    "p_int": $p_int,
+    "p_dec": $p_dec,
+    "p_dbl": $p_dbl,
+    "p_str": $p_str,
+    "p_arr": $p_arr,
+    "p_obj": $p_obj
+  },
+
+  "t2": {
+    "p_null_type": $p_null is null,
+    "p_bool_type": is_boolean($p_bool),
+    "p_int_type": is_number($p_int),
+    "p_dec_type": is_number($p_dec),
+    "p_dbl_type": is_number($p_dbl),
+    "p_str_type": is_string($p_str),
+    "p_arr_type": is_array($p_arr),
+    "p_obj_type": is_object($p_obj)
+  },
+
+  "t3": [ $p_null, $p_bool, $p_int, $p_dec, $p_dbl, $p_str, $p_arr, $p_obj ]
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.2.query.sqlpp
new file mode 100644
index 0000000..6ecefd1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_01/named_01.2.query.sqlpp
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with url encoded request.
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param $p_null:json=null
+// param $p_bool:json=true
+// param $p_int:json=42
+// param $p_dec:json=42.5
+// param $p_dbl:json=42.5e2
+// param $p_str:json="hello"
+// param $p_arr:json=["99",100,{"a":null},null,true]
+// param $p_obj:json={"a":[1,2,3]}
+
+{
+  "t1": {
+    "p_null": $p_null,
+    "p_bool": $p_bool,
+    "p_int": $p_int,
+    "p_dec": $p_dec,
+    "p_dbl": $p_dbl,
+    "p_str": $p_str,
+    "p_arr": $p_arr,
+    "p_obj": $p_obj
+  },
+
+  "t2": {
+    "p_null_type": is_string($p_null),
+    "p_bool_type": is_string($p_bool),
+    "p_int_type": is_string($p_int),
+    "p_dec_type": is_string($p_dec),
+    "p_dbl_type": is_string($p_dbl),
+    "p_str_type": is_string($p_str),
+    "p_arr_type": is_string($p_arr),
+    "p_obj_type": is_string($p_obj)
+  },
+
+  "t3": [ $p_null, $p_bool, $p_int, $p_dec, $p_dbl, $p_str, $p_arr, $p_obj ]
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_02/named_02.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_02/named_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_02/named_02.1.query.sqlpp
new file mode 100644
index 0000000..f2ec9e0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_02/named_02.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with json encoded request
+ * Expected Res : Failure (no value for a named parameter)
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param $p1:json="hello"
+
+$p1 || $p2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_03/named_03.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_03/named_03.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_03/named_03.1.query.sqlpp
new file mode 100644
index 0000000..522b776
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/named_03/named_03.1.query.sqlpp
@@ -0,0 +1,31 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test autogenerated column aliases named statement parameters
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param $p_int:json=42
+// param $p_str:json="hello"
+
+select $p_int, $p_str
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.1.query.sqlpp
new file mode 100644
index 0000000..ef71010
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.1.query.sqlpp
@@ -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.
+ */
+
+/*
+ * Description  : Test positional ($) statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t1": {
+    "p_null": $1,
+    "p_bool": $2,
+    "p_int": $3,
+    "p_dec": $4,
+    "p_dbl": $5,
+    "p_str": $6,
+    "p_arr": $7,
+    "p_obj": $8
+  },
+
+  "t2": {
+    "p_null_type": $1 is null,
+    "p_bool_type": is_boolean($2),
+    "p_int_type": is_number($3),
+    "p_dec_type": is_number($4),
+    "p_dbl_type": is_number($5),
+    "p_str_type": is_string($6),
+    "p_arr_type": is_array($7),
+    "p_obj_type": is_object($8)
+  },
+
+  "t3": [ $1, $2, $3, $4, $5, $6, $7, $8 ]
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.2.query.sqlpp
new file mode 100644
index 0000000..a31a646
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_01/positional_01.2.query.sqlpp
@@ -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.
+ */
+
+/*
+ * Description  : Test positional ($) statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t1": {
+    "p_null": $1,
+    "p_bool": $2,
+    "p_int": $3,
+    "p_dec": $4,
+    "p_dbl": $5,
+    "p_str": $6,
+    "p_arr": $7,
+    "p_obj": $8
+  },
+
+  "t2": {
+    "p_null_type": $1 is null,
+    "p_bool_type": is_boolean($2),
+    "p_int_type": is_number($3),
+    "p_dec_type": is_number($4),
+    "p_dbl_type": is_number($5),
+    "p_str_type": is_string($6),
+    "p_arr_type": is_array($7),
+    "p_obj_type": is_object($8)
+  },
+
+  "t3": [ $1, $2, $3, $4, $5, $6, $7, $8 ]
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.1.query.sqlpp
new file mode 100644
index 0000000..2fa7cde
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.1.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t1": {
+    "p_null": ?,
+    "p_bool": ?,
+    "p_int": ?,
+    "p_dec": ?,
+    "p_dbl": ?,
+    "p_str": ?,
+    "p_arr": ?,
+    "p_obj": ?
+  }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.2.query.sqlpp
new file mode 100644
index 0000000..1d952d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.2.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t2": {
+    "p_null_type": ? is null,
+    "p_bool_type": is_boolean(?),
+    "p_int_type": is_number(?),
+    "p_dec_type": is_number(?),
+    "p_dbl_type": is_number(?),
+    "p_str_type": is_string(?),
+    "p_arr_type": is_array(?),
+    "p_obj_type": is_object(?)
+  }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.3.query.sqlpp
new file mode 100644
index 0000000..13c077b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_02/positional_02.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with json encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t3": [ ?, ?, ?, ?, ?, ?, ?, ? ]
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.1.query.sqlpp
new file mode 100644
index 0000000..1eb6e0f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.1.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t1": {
+    "p_null": ?,
+    "p_bool": ?,
+    "p_int": ?,
+    "p_dec": ?,
+    "p_dbl": ?,
+    "p_str": ?,
+    "p_arr": ?,
+    "p_obj": ?
+  }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.2.query.sqlpp
new file mode 100644
index 0000000..bbacada
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.2.query.sqlpp
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t2": {
+    "p_null_type": ? is null,
+    "p_bool_type": is_boolean(?),
+    "p_int_type": is_number(?),
+    "p_dec_type": is_number(?),
+    "p_dbl_type": is_number(?),
+    "p_str_type": is_string(?),
+    "p_arr_type": is_array(?),
+    "p_obj_type": is_object(?)
+  }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.3.query.sqlpp
new file mode 100644
index 0000000..312278f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_03/positional_02.3.query.sqlpp
@@ -0,0 +1,32 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test positional (?) statement parameters with url encoded request
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/x-www-form-urlencoded
+
+// param args:json=[null, true, 42, 42.5, 42.5e2, "hello", ["99",100,{"a":null},null,true], {"a":[1,2,3]}]
+
+{
+  "t3": [ ?, ?, ?, ?, ?, ?, ?, ? ]
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.1.query.sqlpp
new file mode 100644
index 0000000..279f336
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with json encoded request
+ * Expected Res : Failure (no value for a positional parameter)
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=["hello"]
+
+$1 || $2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.2.query.sqlpp
new file mode 100644
index 0000000..5f7dedc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_04/positional_04.2.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test named statement parameters with json encoded request
+ * Expected Res : Failure (no value for a positional parameter)
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=["a", "b"]
+
+? || ? || ?
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_05/positional_05.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_05/positional_05.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_05/positional_05.1.query.sqlpp
new file mode 100644
index 0000000..89901e7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/statement-params/positional_05/positional_05.1.query.sqlpp
@@ -0,0 +1,30 @@
+/*
+ * 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.
+ */
+
+/*
+ * Description  : Test autogenerated column aliases positional statement parameters
+ * Expected Res : Success
+ * Date         : Jun 2018
+ */
+
+// requesttype=application/json
+
+// param args:json=[3,4]
+
+select $2, $1
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.1.adm
new file mode 100644
index 0000000..7190640
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.1.adm
@@ -0,0 +1 @@
+{ "t1": 7.0, "t2": "hello world world" }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.2.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.2.adm
new file mode 100644
index 0000000..3ed423f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/mixed_01/mixed_01.2.adm
@@ -0,0 +1 @@
+{ "t1": "a b b c c" }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.1.adm
new file mode 100644
index 0000000..2d95320
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.1.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } }, "t2": { "p_null_type": true, "p_bool_type": true, "p_int_type": true, "p_dec_type": true, "p_dbl_type": true, "p_str_type": true, "p_arr_type": true, "p_obj_type": true }, "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.2.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.2.adm
new file mode 100644
index 0000000..e16cae9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_01/named_01.2.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } }, "t2": { "p_null_type": null, "p_bool_type": false, "p_int_type": false, "p_dec_type": false, "p_dbl_type": false, "p_str_type": true, "p_arr_type": false, "p_obj_type": false }, "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_03/named_03.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_03/named_03.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_03/named_03.1.adm
new file mode 100644
index 0000000..e7eb119
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/named_03/named_03.1.adm
@@ -0,0 +1 @@
+{ "$1": 42, "$2": "hello" }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.1.adm
new file mode 100644
index 0000000..aab6f21
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.1.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } }, "t2": { "p_null_type": true, "p_bool_type": true, "p_int_type": true, "p_dec_type": true, "p_dbl_type": true, "p_str_type": true, "p_arr_type": true, "p_obj_type": true }, "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.2.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.2.adm
new file mode 100644
index 0000000..2d95320
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_01/positional_01.2.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } }, "t2": { "p_null_type": true, "p_bool_type": true, "p_int_type": true, "p_dec_type": true, "p_dbl_type": true, "p_str_type": true, "p_arr_type": true, "p_obj_type": true }, "t3": [ null, true, 42, 42.5, 4250.0, "hello", [ "99", 100, { "a": null }, null, true ], { "a": [ 1, 2, 3 ] } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.1.adm
new file mode 100644
index 0000000..f8c0d3e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.1.adm
@@ -0,0 +1 @@
+{ "t1": { "p_null": null, "p_bool": true, "p_int": 42, "p_dec": 42.5, "p_dbl": 4250.0, "p_str": "hello", "p_arr": [ "99", 100, { "a": null }, null, true ], "p_obj": { "a": [ 1, 2, 3 ] } } }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/1bdf8082/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.2.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.2.adm
new file mode 100644
index 0000000..5a2e691
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/statement-params/positional_02/positional_02.2.adm
@@ -0,0 +1 @@
+{ "t2": { "p_null_type": true, "p_bool_type": true, "p_int_type": true, "p_dec_type": true, "p_dbl_type": true, "p_str_type": true, "p_arr_type": true, "p_obj_type": true } }
\ No newline at end of file