You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2017/03/17 02:19:09 UTC

asterixdb git commit: Add query parameter validation.

Repository: asterixdb
Updated Branches:
  refs/heads/master 0ecc68081 -> c3b4e4161


Add query parameter validation.

- Automatically adjust query parameters such as compiler.joinmemory
  compiler.sortmemory, compiler.groupmemory if they're too small;
- Validate if query parameters specified in a query are supported.

Change-Id: Ia9e2df274a0c5cf598da6c37d0241b6b9d35c84d
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1584
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@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/c3b4e416
Tree: http://git-wip-us.apache.org/repos/asf/asterixdb/tree/c3b4e416
Diff: http://git-wip-us.apache.org/repos/asf/asterixdb/diff/c3b4e416

Branch: refs/heads/master
Commit: c3b4e4161b9d204036ca97a610c5e507f946dc91
Parents: 0ecc680
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Thu Mar 16 14:20:05 2017 -0700
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Thu Mar 16 19:18:48 2017 -0700

----------------------------------------------------------------------
 .../asterix/optimizer/base/FuzzyUtils.java      |   4 +-
 .../apache/asterix/api/common/APIFramework.java |  46 ++++++-
 .../unsupported_parameter.1.query.sqlpp         |  22 +++
 .../unsupported_parameter_value.1.query.sqlpp   |  22 +++
 ...pricing_summary_report_parameter.1.ddl.sqlpp |  47 +++++++
 ...cing_summary_report_parameter.2.update.sqlpp |  24 ++++
 ...icing_summary_report_parameter.3.query.sqlpp |  40 ++++++
 ...09_product_type_profit_parameter.1.ddl.sqlpp | 134 +++++++++++++++++++
 ...product_type_profit_parameter.2.update.sqlpp |  38 ++++++
 ..._product_type_profit_parameter.3.query.sqlpp |  48 +++++++
 .../q17_large_gby_variant_parameter.1.ddl.sqlpp | 134 +++++++++++++++++++
 ...7_large_gby_variant_parameter.2.update.sqlpp |  38 ++++++
 ...17_large_gby_variant_parameter.3.query.sqlpp |  39 ++++++
 .../resources/runtimets/testsuite_sqlpp.xml     |  27 ++++
 .../common/exceptions/AsterixException.java     |   8 +-
 .../asterix/common/exceptions/ErrorCode.java    |   1 +
 .../main/resources/asx_errormsg/en.properties   |   1 +
 17 files changed, 663 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java
index 42a9c25..f888ee0 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/FuzzyUtils.java
@@ -40,8 +40,8 @@ public class FuzzyUtils {
     private final static float JACCARD_DEFAULT_SIM_THRESHOLD = .8f;
     private final static int EDIT_DISTANCE_DEFAULT_SIM_THRESHOLD = 1;
 
-    private final static String SIM_FUNCTION_PROP_NAME = "simfunction";
-    private final static String SIM_THRESHOLD_PROP_NAME = "simthreshold";
+    public final static String SIM_FUNCTION_PROP_NAME = "simfunction";
+    public final static String SIM_THRESHOLD_PROP_NAME = "simthreshold";
 
     public final static String JACCARD_FUNCTION_NAME = "jaccard";
     public final static String EDIT_DISTANCE_FUNCTION_NAME = "edit-distance";

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
index 3e722cc..7896424 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
@@ -29,14 +29,15 @@ import java.util.Map;
 import java.util.Random;
 import java.util.Set;
 
-import com.fasterxml.jackson.databind.ObjectMapper;
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
+import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.asterix.common.utils.Job;
 import org.apache.asterix.common.utils.Job.SubmissionMode;
 import org.apache.asterix.compiler.provider.ILangCompilationProvider;
@@ -46,8 +47,10 @@ import org.apache.asterix.dataflow.data.common.ExpressionTypeComputer;
 import org.apache.asterix.dataflow.data.common.MergeAggregationExpressionFactory;
 import org.apache.asterix.dataflow.data.common.MissableTypeComputer;
 import org.apache.asterix.dataflow.data.common.PartialAggregationTypeComputer;
+import org.apache.asterix.external.feed.watch.FeedActivityDetails;
 import org.apache.asterix.formats.base.IDataFormat;
 import org.apache.asterix.jobgen.QueryLogicalExpressionJobGen;
+import org.apache.asterix.lang.aql.statement.SubscribeFeedStatement;
 import org.apache.asterix.lang.common.base.IAstPrintVisitorFactory;
 import org.apache.asterix.lang.common.base.IQueryRewriter;
 import org.apache.asterix.lang.common.base.IReturningStatement;
@@ -56,7 +59,9 @@ import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.rewrites.LangRewritingContext;
 import org.apache.asterix.lang.common.statement.FunctionDecl;
 import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.lang.common.util.FunctionUtil;
 import org.apache.asterix.metadata.declared.MetadataProvider;
+import org.apache.asterix.optimizer.base.FuzzyUtils;
 import org.apache.asterix.runtime.job.listener.JobEventListenerFactory;
 import org.apache.asterix.runtime.utils.AppContextInfo;
 import org.apache.asterix.transaction.management.service.transaction.JobIdFactory;
@@ -94,12 +99,28 @@ import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.control.common.config.OptionTypes;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableSet;
+
 /**
  * Provides helper methods for compilation of a query into a JobSpec and submission
  * to Hyracks through the Hyracks client interface.
  */
 public class APIFramework {
 
+    private static final int MIN_FRAME_LIMIT_FOR_SORT = 3;
+    private static final int MIN_FRAME_LIMIT_FOR_GROUP_BY = 4;
+    private static final int MIN_FRAME_LIMIT_FOR_JOIN = 5;
+
+    // A white list of supported configurable parameters.
+    private static final Set<String> CONFIGURABLE_PARAMETER_NAMES = ImmutableSet.of(
+            CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
+            CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_PARALLELISM_KEY,
+            FunctionUtil.IMPORT_PRIVATE_FUNCTIONS, FuzzyUtils.SIM_FUNCTION_PROP_NAME,
+            FuzzyUtils.SIM_THRESHOLD_PROP_NAME, SubscribeFeedStatement.WAIT_FOR_COMPLETION,
+            FeedActivityDetails.FEED_POLICY_NAME, FeedActivityDetails.COLLECT_LOCATIONS, "inline_with", "hash_merge",
+            "output-record-type");
+
     private final IRewriterFactory rewriterFactory;
     private final IAstPrintVisitorFactory astPrintVisitorFactory;
     private final ILangExpressionToPlanTranslatorFactory translatorFactory;
@@ -204,12 +225,13 @@ public class APIFramework {
         CompilerProperties compilerProperties = AppContextInfo.INSTANCE.getCompilerProperties();
         int frameSize = compilerProperties.getFrameSize();
         Map<String, String> querySpecificConfig = metadataProvider.getConfig();
+        validateConfig(querySpecificConfig); // Validates the user-overridden query parameters.
         int sortFrameLimit = getFrameLimit(querySpecificConfig.get(CompilerProperties.COMPILER_SORTMEMORY_KEY),
-                compilerProperties.getSortMemorySize(), frameSize);
+                compilerProperties.getSortMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_SORT);
         int groupFrameLimit = getFrameLimit(querySpecificConfig.get(CompilerProperties.COMPILER_GROUPMEMORY_KEY),
-                compilerProperties.getGroupMemorySize(), frameSize);
+                compilerProperties.getGroupMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_GROUP_BY);
         int joinFrameLimit = getFrameLimit(querySpecificConfig.get(CompilerProperties.COMPILER_JOINMEMORY_KEY),
-                compilerProperties.getJoinMemorySize(), frameSize);
+                compilerProperties.getJoinMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_JOIN);
         OptimizationConfUtil.getPhysicalOptimizationConfig().setFrameSize(frameSize);
         OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalSort(sortFrameLimit);
         OptimizationConfUtil.getPhysicalOptimizationConfig().setMaxFramesExternalGroupBy(groupFrameLimit);
@@ -435,11 +457,14 @@ public class APIFramework {
     }
 
     // Gets the frame limit.
-    private int getFrameLimit(String parameter, long memBudgetInConfiguration, int frameSize) {
+    private int getFrameLimit(String parameter, long memBudgetInConfiguration, int frameSize, int minFrameLimit)
+            throws AlgebricksException {
         IOptionType<Long> longBytePropertyInterpreter = OptionTypes.LONG_BYTE_UNIT;
         long memBudget =
                 parameter == null ? memBudgetInConfiguration : longBytePropertyInterpreter.parse(parameter);
-        return (int) (memBudget / frameSize);
+        int frameLimit = (int) (memBudget / frameSize);
+        // Sets the frame limit to the minimum frame limit if the caculated frame limit is too small.
+        return Math.max(frameLimit, minFrameLimit);
     }
 
     // Gets the parallelism parameter.
@@ -447,4 +472,13 @@ public class APIFramework {
         IOptionType<Integer> integerIPropertyInterpreter = OptionTypes.INTEGER;
         return parameter == null ? parallelismInConfiguration : integerIPropertyInterpreter.parse(parameter);
     }
+
+    // Validates if the query contains unsupported query parameters.
+    private void validateConfig(Map<String, String> config) throws AlgebricksException {
+        for (String parameterName : config.keySet()) {
+            if (!CONFIGURABLE_PARAMETER_NAMES.contains(parameterName)) {
+                throw AsterixException.create(ErrorCode.COMPILATION_UNSUPPORTED_QUERY_PARAMETER, parameterName);
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/unsupported_parameter/unsupported_parameter.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/unsupported_parameter/unsupported_parameter.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/unsupported_parameter/unsupported_parameter.1.query.sqlpp
new file mode 100644
index 0000000..b0c83c4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/unsupported_parameter/unsupported_parameter.1.query.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.
+ */
+
+SET `compiler.joinmem` "4MB"
+
+SELECT 1;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/unsupported_parameter_value/unsupported_parameter_value.1.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/unsupported_parameter_value/unsupported_parameter_value.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/unsupported_parameter_value/unsupported_parameter_value.1.query.sqlpp
new file mode 100644
index 0000000..3fb33b8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/unsupported_parameter_value/unsupported_parameter_value.1.query.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.
+ */
+
+SET `compiler.joinmemory` "4LS"
+
+SELECT 1;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.1.ddl.sqlpp
new file mode 100644
index 0000000..92698ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.1.ddl.sqlpp
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+}
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.2.update.sqlpp
new file mode 100644
index 0000000..5fe734c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.3.query.sqlpp
new file mode 100644
index 0000000..3124274
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q01_pricing_summary_report_parameter/q01_pricing_summary_report_parameter.3.query.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * 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.
+ */
+
+USE tpch;
+
+SET `compiler.groupmemory` "-10GB"
+SET `compiler.sortmemory` "-7GB"
+
+SELECT  l_returnflag,
+        l_linestatus,
+        sum(l_quantity) AS sum_qty,
+        sum(l_extendedprice) AS sum_base_price,
+        sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
+        sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
+        avg(l_quantity) AS ave_qty,
+        avg(l_extendedprice) AS ave_price,
+        avg(l_discount) AS ave_disc,
+        count(1) AS count_order
+FROM  LineItem
+WHERE l_shipdate <= '1998-09-02'
+/* +hash */
+GROUP BY l_returnflag, l_linestatus
+ORDER BY l_returnflag, l_linestatus
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.1.ddl.sqlpp
new file mode 100644
index 0000000..0db8d36
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.1.ddl.sqlpp
@@ -0,0 +1,134 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : integer,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+}
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+}
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+}
+
+create type tpch.SupplierType as
+ closed {
+  s_suppkey : integer,
+  s_name : string,
+  s_address : string,
+  s_nationkey : integer,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+}
+
+create type tpch.NationType as
+ closed {
+  n_nationkey : integer,
+  n_name : string,
+  n_regionkey : integer,
+  n_comment : string
+}
+
+create type tpch.RegionType as
+ closed {
+  r_regionkey : integer,
+  r_name : string,
+  r_comment : string
+}
+
+create type tpch.PartType as
+ closed {
+  p_partkey : integer,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : integer,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+}
+
+create type tpch.PartSuppType as
+ closed {
+  ps_partkey : integer,
+  ps_suppkey : integer,
+  ps_availqty : integer,
+  ps_supplycost : double,
+  ps_comment : string
+}
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create  dataset Supplier(SupplierType) primary key s_suppkey;
+
+create  dataset Region(RegionType) primary key r_regionkey;
+
+create  dataset Nation(NationType) primary key n_nationkey;
+
+create  dataset Part(PartType) primary key p_partkey;
+
+create  dataset Partsupp(PartSuppType) primary key ps_partkey,ps_suppkey;
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.2.update.sqlpp
new file mode 100644
index 0000000..e72e451
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.2.update.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Supplier using localfs ((`path`=`asterix_nc1://data/tpch0.001/supplier.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Region using localfs ((`path`=`asterix_nc1://data/tpch0.001/region.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Nation using localfs ((`path`=`asterix_nc1://data/tpch0.001/nation.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Part using localfs ((`path`=`asterix_nc1://data/tpch0.001/part.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Partsupp using localfs ((`path`=`asterix_nc1://data/tpch0.001/partsupp.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Customer using localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.3.query.sqlpp
new file mode 100644
index 0000000..4ab8816
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q09_product_type_profit_parameter/q09_product_type_profit_parameter.3.query.sqlpp
@@ -0,0 +1,48 @@
+/*
+ * 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.
+ */
+
+USE tpch;
+
+SET `compiler.joinmemory` "-32GB"
+
+SELECT nation, o_year, sum(amount) AS sum_profit
+FROM
+  (
+    SELECT   n_name AS nation,
+             get_year(o_orderdate) AS o_year,
+             l_extendedprice * (1 - l_discount) -  ps_supplycost * l_quantity AS amount
+    FROM
+      Orders JOIN
+      (
+       SELECT l_extendedprice, l_discount, l_quantity, l_orderkey, n_name, ps_supplycost
+       FROM Part JOIN
+         (SELECT l_extendedprice, l_discount, l_quantity, l_partkey, l_orderkey, n_name, ps_supplycost
+          FROM Partsupp join
+            (SELECT l_suppkey, l_extendedprice, l_discount, l_quantity, l_partkey, l_orderkey, n_name
+             FROM
+               (SELECT s_suppkey, n_name
+                FROM Nation JOIN Supplier ON n_nationkey = s_nationkey
+               ) s1 JOIN LineItem ON s_suppkey = l_suppkey
+            ) l1 ON ps_suppkey = l_suppkey AND ps_partkey = l_partkey
+         ) l2 ON contains(p_name,'green') AND p_partkey = l_partkey
+     ) l3 ON o_orderkey = l_orderkey
+  ) profit
+GROUP BY nation, o_year
+ORDER BY nation, o_year desc;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.1.ddl.sqlpp
new file mode 100644
index 0000000..6cbdf11
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.1.ddl.sqlpp
@@ -0,0 +1,134 @@
+/*
+ * 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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : bigint,
+  l_partkey : bigint,
+  l_suppkey : bigint,
+  l_linenumber : bigint,
+  l_quantity : bigint,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+}
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+}
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+}
+
+create type tpch.SupplierType as
+ closed {
+  s_suppkey : bigint,
+  s_name : string,
+  s_address : string,
+  s_nationkey : bigint,
+  s_phone : string,
+  s_acctbal : double,
+  s_comment : string
+}
+
+create type tpch.NationType as
+ closed {
+  n_nationkey : bigint,
+  n_name : string,
+  n_regionkey : bigint,
+  n_comment : string
+}
+
+create type tpch.RegionType as
+ closed {
+  r_regionkey : bigint,
+  r_name : string,
+  r_comment : string
+}
+
+create type tpch.PartType as
+ closed {
+  p_partkey : bigint,
+  p_name : string,
+  p_mfgr : string,
+  p_brand : string,
+  p_type : string,
+  p_size : bigint,
+  p_container : string,
+  p_retailprice : double,
+  p_comment : string
+}
+
+create type tpch.PartSuppType as
+ closed {
+  ps_partkey : bigint,
+  ps_suppkey : bigint,
+  ps_availqty : bigint,
+  ps_supplycost : double,
+  ps_comment : string
+}
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create  dataset Supplier(SupplierType) primary key s_suppkey;
+
+create  dataset Region(RegionType) primary key r_regionkey;
+
+create  dataset Nation(NationType) primary key n_nationkey;
+
+create  dataset Part(PartType) primary key p_partkey;
+
+create  dataset Partsupp(PartSuppType) primary key ps_partkey,ps_suppkey;
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.2.update.sqlpp
new file mode 100644
index 0000000..e72e451
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.2.update.sqlpp
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Supplier using localfs ((`path`=`asterix_nc1://data/tpch0.001/supplier.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Region using localfs ((`path`=`asterix_nc1://data/tpch0.001/region.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Nation using localfs ((`path`=`asterix_nc1://data/tpch0.001/nation.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Part using localfs ((`path`=`asterix_nc1://data/tpch0.001/part.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Partsupp using localfs ((`path`=`asterix_nc1://data/tpch0.001/partsupp.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+load  dataset Customer using localfs ((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`));
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.3.query.sqlpp
new file mode 100644
index 0000000..b16b388
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/q17_large_gby_variant_parameter/q17_large_gby_variant_parameter.3.query.sqlpp
@@ -0,0 +1,39 @@
+/*
+ * 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.
+ */
+
+use tpch;
+
+SET `compiler.groupmemory` "-10GB"
+
+SELECT l_partkey t_partkey,
+       count(1) t_count,
+       0.2 * avg(l_quantity) t_avg_quantity,
+       max(l_suppkey) t_max_suppkey,
+       max(l_linenumber) t_max_linenumber,
+       avg(l_extendedprice) t_avg_extendedprice,
+       avg(l_discount) t_avg_discount,
+       avg(l_tax) t_avg_tax,
+       max(l_shipdate) t_max_shipdate,
+       min(l_commitdate) t_min_commitdate,
+       min(l_receiptdate) t_min_receiptdate,
+       max(l_comment) t_max_comment
+FROM  LineItem
+GROUP BY l_partkey
+ORDER BY l_partkey
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index bcb95aa..557e802 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3282,6 +3282,18 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="unsupported_parameter">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>Query parameter compiler.joinmem is not supported</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="unsupported_parameter_value">
+        <output-dir compare="Text">none</output-dir>
+        <expected-error>The given string: 4LS is not a byte unit string (e.g., 320KB or 1024)</expected-error>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="uuid">
         <output-dir compare="Text">uuid</output-dir>
       </compilation-unit>
@@ -6864,6 +6876,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q17_large_gby_variant_parameter">
+        <output-dir compare="Text">q17_large_gby_variant</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
       <compilation-unit name="q18_large_volume_customer">
         <output-dir compare="Text">q18_large_volume_customer</output-dir>
       </compilation-unit>
@@ -6889,6 +6906,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q01_pricing_summary_report_parameter">
+        <output-dir compare="Text">q01_pricing_summary_report_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
       <compilation-unit name="q20_potential_part_promotion">
         <output-dir compare="Text">q20_potential_part_promotion</output-dir>
       </compilation-unit>
@@ -6944,6 +6966,11 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="tpch-sql-sugar">
+      <compilation-unit name="q09_product_type_profit_parameter">
+        <output-dir compare="Text">q09_product_type_profit_nt</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="tpch-sql-sugar">
       <compilation-unit name="query-issue562">
         <output-dir compare="Text">query-issue562</output-dir>
       </compilation-unit>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
index 7a73705..7cb7d8a 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/AsterixException.java
@@ -18,10 +18,10 @@
  */
 package org.apache.asterix.common.exceptions;
 
-import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
-
 import java.io.Serializable;
 
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+
 public class AsterixException extends AlgebricksException {
     private static final long serialVersionUID = 1L;
 
@@ -34,6 +34,10 @@ public class AsterixException extends AlgebricksException {
 
     }
 
+    public static AsterixException create(int errorCode, Serializable... params) {
+        return new AsterixException(errorCode, params);
+    }
+
     public AsterixException(Throwable cause) {
         super(cause);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
index 96ac491..6748287 100644
--- a/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
+++ b/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
@@ -85,6 +85,7 @@ public class ErrorCode {
     public static final int COMPILATION_AQLPLUS_NO_SUCH_JOIN_TYPE = 1025;
     public static final int COMPILATION_FUNC_EXPRESSION_CANNOT_UTILIZE_INDEX = 1026;
     public static final int COMPILATION_DATASET_TYPE_DOES_NOT_HAVE_PRIMARY_INDEX = 1027;
+    public static final int COMPILATION_UNSUPPORTED_QUERY_PARAMETER = 1028;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/c3b4e416/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
index a0bfab9..c4698df 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -71,6 +71,7 @@
 1025 = There is no such join type in AQL+
 1026 = The given function expression %1$s cannot utilize index
 1027 = Dataset of type %1$s doesn't have a primary index
+1028 = Query parameter %1$s is not supported
 
 # Feed Errors
 3001 = Illegal state.