You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@asterixdb.apache.org by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org> on 2018/07/19 05:47:39 UTC

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Dmitry Lychagin has uploaded a new change for review.

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

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................

[ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

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

Details:
- Fixed NPE in optimizer when limit/offset is non-integer
- Allow float/double values that are integers in limit/offset
- Make consistent limit/offset value types (integer) between
  runtime and optimizer rules
- Add internal function treat-as-integer() that returns integer
  if input number conforms to the integer type or fails otherwise

Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
23 files changed, 612 insertions(+), 117 deletions(-)


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

diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 134c96f..85d1c29 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -182,13 +182,14 @@
         normalization.add(new IntroduceDynamicTypeCastRule());
         normalization.add(new IntroduceDynamicTypeCastForExternalFunctionRule());
         normalization.add(new IntroduceEnforcedListTypeRule());
+        // Perform constant folding before common expression extraction
+        normalization.add(new ConstantFoldingRule(appCtx));
         normalization.add(new ExtractCommonExpressionsRule());
 
         // Let PushAggFuncIntoStandaloneAggregateRule run after ExtractCommonExpressionsRule
         // so that PushAggFunc can happen in fewer places.
         normalization.add(new PushAggFuncIntoStandaloneAggregateRule());
         normalization.add(new ListifyUnnestingFunctionRule());
-        normalization.add(new ConstantFoldingRule(appCtx));
         normalization.add(new RemoveRedundantSelectRule());
         normalization.add(new UnnestToDataScanRule());
         normalization.add(new MetaFunctionToMetaVariableRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
index 56dc675..e64889d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
@@ -18,12 +18,15 @@
  */
 package org.apache.asterix.optimizer.rules;
 
-import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -89,50 +92,19 @@
         PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
         LimitOperator limitOp = (LimitOperator) opRef.getValue();
         OrderOperator orderOp = (OrderOperator) opRef2.getValue();
-        long topK = -1;
 
         // We don't push-down LIMIT into in-memory sort.
         if (orderOp.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT) {
             return false;
         }
 
-        // Get the LIMIT constant
-        if (limitOp.getMaxObjects().getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-            // Currently, we support LIMIT with a constant value.
-            topK = AccessMethodUtils.getInt64Constant(limitOp.getMaxObjects());
-            // If topK is huge, there is no reason to use topK sort module
-            // since the original external sort's performance might be better.
-            if (topK > Integer.MAX_VALUE) {
-                return false;
-            }
-            if (topK < 0) {
-                topK = 0;
-            }
-        } else {
+        Integer topK = getOutputLimit(limitOp);
+        if (topK == null) {
             return false;
         }
 
-        // Get the offset constant if there is one. If one presents, then topK = topK + offset.
-        // This is because we can't apply offset to the external sort.
-        // Final topK will be applied through LIMIT.
-        if (limitOp.getOffset().getValue() != null) {
-            if (limitOp.getOffset().getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                long offset = AccessMethodUtils.getInt64Constant(limitOp.getOffset());
-                if (offset < 0) {
-                    offset = 0;
-                }
-                // Check the overflow case.
-                if (offset >= Integer.MAX_VALUE - topK) {
-                    return false;
-                }
-                topK += offset;
-            } else {
-                return false;
-            }
-        }
-
         // Create the new ORDER operator, set the topK value, and replace the current one.
-        OrderOperator newOrderOp = new OrderOperator(orderOp.getOrderExpressions(), (int) topK);
+        OrderOperator newOrderOp = new OrderOperator(orderOp.getOrderExpressions(), topK);
         newOrderOp.setSourceLocation(orderOp.getSourceLocation());
         newOrderOp.setPhysicalOperator(
                 new StableSortPOperator(physicalOptimizationConfig.getMaxFramesExternalSort(), newOrderOp.getTopK()));
@@ -146,4 +118,38 @@
         return true;
     }
 
+    static Integer getOutputLimit(LimitOperator limitOp) {
+        // Currently, we support LIMIT with a constant value.
+        ILogicalExpression maxObjectsExpr = limitOp.getMaxObjects().getValue();
+        IAObject maxObjectsValue = ConstantExpressionUtil.getConstantIaObject(maxObjectsExpr, ATypeTag.INTEGER);
+        if (maxObjectsValue == null) {
+            return null;
+        }
+        int topK = ((AInt32) maxObjectsValue).getIntegerValue();
+        if (topK < 0) {
+            topK = 0;
+        }
+
+        // Get the offset constant if there is one. If one presents, then topK = topK + offset.
+        // This is because we can't apply offset to the external sort.
+        // Final topK will be applied through LIMIT.
+        ILogicalExpression offsetExpr = limitOp.getOffset().getValue();
+        if (offsetExpr != null) {
+            IAObject offsetValue = ConstantExpressionUtil.getConstantIaObject(offsetExpr, ATypeTag.INTEGER);
+            if (offsetValue == null) {
+                return null;
+            }
+            int offset = ((AInt32) offsetValue).getIntegerValue();
+            if (offset < 0) {
+                offset = 0;
+            }
+            // Check the overflow case.
+            if (offset >= Integer.MAX_VALUE - topK) {
+                return null;
+            }
+            topK += offset;
+        }
+
+        return topK;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
index 921d231..5427ca6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
@@ -25,7 +25,6 @@
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
-import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -67,7 +66,7 @@
         }
         context.addToDontApplySet(this, op);
 
-        Long outputLimit = getOutputLimit((LimitOperator) op);
+        Integer outputLimit = PushLimitIntoOrderByRule.getOutputLimit((LimitOperator) op);
         if (outputLimit == null) {
             // we cannot push if limit is not constant
             return false;
@@ -77,7 +76,7 @@
         if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
             childOp = childOp.getValue().getInputs().get(0);
         }
-        boolean changed = false;
+        boolean changed;
         if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.SELECT) {
             changed = rewriteSelect(childOp, outputLimit);
         } else {
@@ -89,21 +88,7 @@
         return changed;
     }
 
-    private Long getOutputLimit(LimitOperator limit) {
-        if (limit.getMaxObjects().getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-            return null;
-        }
-        long outputLimit = AccessMethodUtils.getInt64Constant(limit.getMaxObjects());
-        if (limit.getOffset() != null && limit.getOffset().getValue() != null) {
-            if (limit.getOffset().getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return null;
-            }
-            outputLimit += AccessMethodUtils.getInt64Constant(limit.getOffset());
-        }
-        return outputLimit;
-    }
-
-    private boolean rewriteSelect(Mutable<ILogicalOperator> op, long outputLimit) throws AlgebricksException {
+    private boolean rewriteSelect(Mutable<ILogicalOperator> op, int outputLimit) {
         SelectOperator select = (SelectOperator) op.getValue();
         Set<LogicalVariable> selectedVariables = new HashSet<>();
         select.getCondition().getValue().getUsedVariables(selectedVariables);
@@ -131,7 +116,7 @@
         return changed;
     }
 
-    private boolean setLimitForScanOrUnnestMap(ILogicalOperator op, long outputLimit) throws AlgebricksException {
+    private boolean setLimitForScanOrUnnestMap(ILogicalOperator op, int outputLimit) {
         if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
             DataSourceScanOperator scan = (DataSourceScanOperator) op;
             if (isScanPushable(scan, Collections.emptySet())) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 18a6597..5410a94 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -1085,38 +1085,27 @@
             ILogicalExpression e = p.first;
             // now look at the operator
             if (i < nOps) {
-                if (OperatorExpr.opIsComparison(ops.get(i))) {
-                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i), sourceLoc);
+                OperatorType opType = ops.get(i);
+                boolean isCmpOp = OperatorExpr.opIsComparison(opType);
+                AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opType, sourceLoc);
 
-                    // chain the operators
-                    if (i == 0) {
-                        c.getArguments().add(new MutableObject<>(e));
-                        currExpr = c;
-                        if (op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.LEFT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
-                    } else {
-                        currExpr.getArguments().add(new MutableObject<>(e));
-                        c.getArguments().add(new MutableObject<>(currExpr));
-                        currExpr = c;
-                        if (i == 1 && op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.RIGHT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
+                // chain the operators
+                if (i == 0) {
+                    f.getArguments().add(new MutableObject<>(e));
+                    currExpr = f;
+                    if (isCmpOp && op.isBroadcastOperand(i)) {
+                        BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                        bcast.setObject(BroadcastSide.LEFT);
+                        f.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                     }
                 } else {
-                    AbstractFunctionCallExpression f =
-                            createFunctionCallExpressionForBuiltinOperator(ops.get(i), sourceLoc);
-                    if (i == 0) {
-                        f.getArguments().add(new MutableObject<>(e));
-                        currExpr = f;
-                    } else {
-                        currExpr.getArguments().add(new MutableObject<>(e));
-                        f.getArguments().add(new MutableObject<>(currExpr));
-                        currExpr = f;
+                    currExpr.getArguments().add(new MutableObject<>(e));
+                    f.getArguments().add(new MutableObject<>(currExpr));
+                    currExpr = f;
+                    if (isCmpOp && i == 1 && op.isBroadcastOperand(i)) {
+                        BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                        bcast.setObject(BroadcastSide.RIGHT);
+                        f.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                     }
                 }
             } else { // don't forget the last expression...
@@ -1359,53 +1348,52 @@
     public Pair<ILogicalOperator, LogicalVariable> visit(LimitClause lc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
         SourceLocation sourceLoc = lc.getSourceLocation();
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = langExprToAlgExpression(lc.getLimitExpr(), tupSource);
         LimitOperator opLim;
+
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = langExprToAlgExpression(lc.getLimitExpr(), tupSource);
+        AbstractFunctionCallExpression maxObjectsExpr =
+                createFunctionCallExpression(BuiltinFunctions.TREAT_AS_INTEGER, lc.getLimitExpr().getSourceLocation());
+        maxObjectsExpr.getArguments().add(new MutableObject<>(p1.first));
+
         Expression offset = lc.getOffset();
         if (offset != null) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = langExprToAlgExpression(offset, p1.second);
-            opLim = new LimitOperator(p1.first, p2.first);
+            AbstractFunctionCallExpression offsetExpr =
+                    createFunctionCallExpression(BuiltinFunctions.TREAT_AS_INTEGER, lc.getOffset().getSourceLocation());
+            offsetExpr.getArguments().add(new MutableObject<>(p2.first));
+            opLim = new LimitOperator(maxObjectsExpr, offsetExpr);
             opLim.getInputs().add(p2.second);
             opLim.setSourceLocation(sourceLoc);
         } else {
-            opLim = new LimitOperator(p1.first);
+            opLim = new LimitOperator(maxObjectsExpr);
             opLim.getInputs().add(p1.second);
             opLim.setSourceLocation(sourceLoc);
         }
         return new Pair<>(opLim, null);
     }
 
-    protected AbstractFunctionCallExpression createComparisonExpression(OperatorType t, SourceLocation sourceLoc) {
-        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-        IFunctionInfo finfo = FunctionUtil.getFunctionInfo(fi);
-        ScalarFunctionCallExpression callExpr = new ScalarFunctionCallExpression(finfo);
-        callExpr.setSourceLocation(sourceLoc);
-        return callExpr;
-    }
-
-    private static FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-        switch (t) {
-            case EQ:
-                return AlgebricksBuiltinFunctions.EQ;
-            case NEQ:
-                return AlgebricksBuiltinFunctions.NEQ;
-            case GT:
-                return AlgebricksBuiltinFunctions.GT;
-            case GE:
-                return AlgebricksBuiltinFunctions.GE;
-            case LT:
-                return AlgebricksBuiltinFunctions.LT;
-            case LE:
-                return AlgebricksBuiltinFunctions.LE;
-            default:
-                throw new IllegalStateException();
-        }
-    }
-
-    protected AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t,
+    private static AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t,
             SourceLocation sourceLoc) throws CompilationException {
         FunctionIdentifier fid;
         switch (t) {
+            case EQ:
+                fid = AlgebricksBuiltinFunctions.EQ;
+                break;
+            case NEQ:
+                fid = AlgebricksBuiltinFunctions.NEQ;
+                break;
+            case GT:
+                fid = AlgebricksBuiltinFunctions.GT;
+                break;
+            case GE:
+                fid = AlgebricksBuiltinFunctions.GE;
+                break;
+            case LT:
+                fid = AlgebricksBuiltinFunctions.LT;
+                break;
+            case LE:
+                fid = AlgebricksBuiltinFunctions.LE;
+                break;
             case PLUS:
                 fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
                 break;
@@ -1440,6 +1428,11 @@
                 throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Operator " + t + " is not yet implemented");
         }
+        return createFunctionCallExpression(fid, sourceLoc);
+    }
+
+    private static AbstractFunctionCallExpression createFunctionCallExpression(FunctionIdentifier fid,
+            SourceLocation sourceLoc) {
         ScalarFunctionCallExpression callExpr = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fid));
         callExpr.setSourceLocation(sourceLoc);
         return callExpr;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
new file mode 100644
index 0000000..c8b2b7a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 expressions returning floating point numbers
+ *                 : in limit/offset
+ * Expected Result : Success
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 7.5/2.5 offset 5/2.5
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
new file mode 100644
index 0000000..ce55ac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 5.5/2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
new file mode 100644
index 0000000..e479296
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 1 offset 3.5/2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
new file mode 100644
index 0000000..b870b76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit "2"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
new file mode 100644
index 0000000..3e3bfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 2 offset true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
new file mode 100644
index 0000000..d783bb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 overflow error in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 9999999999
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
new file mode 100644
index 0000000..991f189
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 overflow error in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 1 offset 8888888888
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
new file mode 100644
index 0000000..5673992
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 test;
+
+explain select element c
+from  LineItem as c
+where (c.l_suppkey < 150)
+limit 7.5/1.5 offset 7.5/1.5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
new file mode 100644
index 0000000..fc9d42f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+  select element paper
+  from  DBLP1 as paper
+  limit 7.5/1.5 offset 7.5/1.5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
new file mode 100644
index 0000000..af64121
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
@@ -0,0 +1,3 @@
+3
+4
+5
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
new file mode 100644
index 0000000..d070b2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$c])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$14(ASC), $$15(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$19, $$20, 2, $$19, $$20, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$19) (ASC, $$20)
+                  -- STABLE_SORT [$$19(ASC), $$20(ASC)]  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$19, $$20])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$17, TRUE, FALSE, FALSE)
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              assign [$$17] <- [150]
+                              -- ASSIGN  |PARTITIONED|
+                                empty-tuple-source
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
new file mode 100644
index 0000000..ee3e565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$paper])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$12, $$paper] <- test.DBLP1 limit 10
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  empty-tuple-source
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
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 bc3b226..c96cc12 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -10077,11 +10077,28 @@
   </test-group>
   <test-group name="limit">
     <test-case FilePath="limit">
+      <compilation-unit name="limit_type_01">
+        <output-dir compare="Text">limit_type_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="limit_type_02">
+        <output-dir compare="Text">limit_type_01</output-dir>
+        <expected-error>ASX0038: Expected integer value, got 2.75 (in line 28, at column 10)</expected-error>
+        <expected-error>ASX0038: Expected integer value, got 1.75 (in line 28, at column 19)</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type string (in line 28, at column 7)</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type boolean (in line 28, at column 16)</expected-error>
+        <expected-error>ASX0021: Source value 9999999999 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+        <expected-error>ASX0021: Source value 8888888888 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
       <compilation-unit name="push-limit-to-primary-scan">
         <output-dir compare="Text">push-limit-to-primary-scan</output-dir>
       </compilation-unit>
     </test-case>
-     <test-case FilePath="limit">
+    <test-case FilePath="limit">
        <compilation-unit name="push-limit-to-primary-scan-select">
          <output-dir compare="Text">push-limit-to-primary-scan-select</output-dir>
        </compilation-unit>
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 4358eac..eb84da1 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
@@ -78,6 +78,7 @@
     public static final int UNSUPPORTED_MULTIPLE_STATEMENTS = 35;
     public static final int CANNOT_COMPARE_COMPLEX = 36;
     public static final int TYPE_MISMATCH_GENERIC = 37;
+    public static final int INTEGER_VALUE_EXPECTED = 38;
 
     public static final int UNSUPPORTED_JRE = 100;
 
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 5227e4b..8fd4b60 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -72,6 +72,7 @@
 34 = Division by zero.
 35 = Unsupported multiple statements.
 36 = Cannot compare non-primitive values
+38 = Expected integer value, got %1$s
 
 100 = Unsupported JRE: %1$s
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index a373c3b..795458f 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -118,6 +118,7 @@
 import org.apache.asterix.om.typecomputer.impl.ToDoubleTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ToNumberTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ToObjectTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.TreatAsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryBinaryInt64TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryMinusTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryStringInt64TypeComputer;
@@ -1079,6 +1080,9 @@
     public static final FunctionIdentifier TO_STRING =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "to-string", 1);
 
+    public static final FunctionIdentifier TREAT_AS_INTEGER =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "treat-as-integer", 1);
+
     public static final FunctionIdentifier EXTERNAL_LOOKUP =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "external-lookup", FunctionIdentifier.VARARGS);
 
@@ -1274,6 +1278,8 @@
         addFunction(TO_OBJECT, ToObjectTypeComputer.INSTANCE, true);
         addFunction(TO_STRING, AStringTypeComputer.INSTANCE, true);
 
+        addPrivateFunction(TREAT_AS_INTEGER, TreatAsTypeComputer.INSTANCE_INTEGER, true);
+
         addFunction(IF_INF, IfNanOrInfTypeComputer.INSTANCE, true);
         addFunction(IF_MISSING, IfMissingTypeComputer.INSTANCE, true);
         addFunction(IF_MISSING_OR_NULL, IfMissingOrNullTypeComputer.INSTANCE, true);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
new file mode 100644
index 0000000..1a5861b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
@@ -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.
+ */
+
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.exceptions.TypeMismatchException;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class TreatAsTypeComputer extends AbstractResultTypeComputer {
+    public static final TreatAsTypeComputer INSTANCE_INTEGER = new TreatAsTypeComputer(BuiltinType.AINT32);
+
+    private final IAType type;
+
+    private TreatAsTypeComputer(IAType type) {
+        this.type = type;
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        if (ATypeHierarchy.isCompatible(inputType.getTypeTag(), type.getTypeTag())) {
+            return type;
+        } else {
+            throw new TypeMismatchException(expr.getSourceLocation(), inputType.getTypeTag(), type.getTypeTag());
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
new file mode 100644
index 0000000..bb0193d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class TreatAsIntegerDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new TreatAsIntegerDescriptor();
+        }
+    };
+
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+
+                final IScalarEvaluator inputEval = args[0].createScalarEvaluator(ctx);
+                final IPointable inputArg = new VoidPointable();
+                final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                final DataOutput out = resultStorage.getDataOutput();
+                final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+                @SuppressWarnings("unchecked")
+                final ISerializerDeserializer<AInt32> int32Ser =
+                        SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+
+                return new IScalarEvaluator() {
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        inputEval.evaluate(tuple, inputArg);
+
+                        int intValue;
+                        byte[] bytes = inputArg.getByteArray();
+                        int startOffset = inputArg.getStartOffset();
+                        ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[startOffset]];
+                        switch (tt) {
+                            case TINYINT:
+                            case SMALLINT:
+                            case INTEGER:
+                            case BIGINT:
+                                intValue = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes,
+                                        startOffset, true);
+                                break;
+                            case FLOAT:
+                            case DOUBLE:
+                                double doubleValue =
+                                        ATypeHierarchy.getDoubleValue(getIdentifier().getName(), 0, bytes, startOffset);
+                                intValue = asInt(doubleValue);
+                                break;
+                            default:
+                                throw new TypeMismatchException(sourceLoc, bytes[startOffset],
+                                        ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                                        ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                                        ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                        }
+
+                        resultStorage.reset();
+                        aInt32.setValue(intValue);
+                        int32Ser.serialize(aInt32, out);
+                        result.set(resultStorage);
+                    }
+
+                    private int asInt(double d) throws HyracksDataException {
+                        if (Double.isFinite(d)) {
+                            long v = (long) d;
+                            if (v == d && Integer.MIN_VALUE <= v && v <= Integer.MAX_VALUE) {
+                                return (int) v;
+                            }
+                        }
+                        throw new RuntimeDataException(ErrorCode.INTEGER_VALUE_EXPECTED, sourceLoc, d);
+                    }
+                };
+            }
+        };
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.TREAT_AS_INTEGER;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 099d563..e81d265 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -271,6 +271,7 @@
 import org.apache.asterix.runtime.evaluators.functions.ToNumberDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ToObjectDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ToStringDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.TreatAsIntegerDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.UUIDDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.binary.BinaryConcatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.binary.BinaryLengthDescriptor;
@@ -770,6 +771,8 @@
         fc.addGenerated(ToObjectDescriptor.FACTORY);
         fc.addGenerated(ToStringDescriptor.FACTORY);
 
+        fc.addGenerated(TreatAsIntegerDescriptor.FACTORY);
+
         // Cast function
         fc.addGenerated(CastTypeDescriptor.FACTORY);
         fc.addGenerated(CastTypeLaxDescriptor.FACTORY);

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

Gerrit-MessageType: newchange
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4575/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4: Integration-Tests-1

Integration Tests Failed

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3967/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/263/ (11/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/285/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3991/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/309/ (11/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Analytics Compatibility Compilation Successful
https://goo.gl/oTMtJ7 : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2041/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/4477/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has submitted this change and it was merged.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


[ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

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

Details:
- Fixed NPE in optimizer when limit/offset is non-integer
- Allow float/double values that are integers in limit/offset
- Make consistent limit/offset value types (integer) between
  runtime and optimizer rules
- Add internal function treat-as-integer() that returns integer
  if input number conforms to the integer type or fails otherwise

Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Reviewed-on: https://asterix-gerrit.ics.uci.edu/2788
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Contrib: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
23 files changed, 613 insertions(+), 117 deletions(-)

Approvals:
  Till Westmann: Looks good to me, approved
  Jenkins: Verified; ; Verified

Objections:
  Anon. E. Moose #1000171: Violations found
  Jenkins: Violations found



diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
index 134c96f..85d1c29 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
@@ -182,13 +182,14 @@
         normalization.add(new IntroduceDynamicTypeCastRule());
         normalization.add(new IntroduceDynamicTypeCastForExternalFunctionRule());
         normalization.add(new IntroduceEnforcedListTypeRule());
+        // Perform constant folding before common expression extraction
+        normalization.add(new ConstantFoldingRule(appCtx));
         normalization.add(new ExtractCommonExpressionsRule());
 
         // Let PushAggFuncIntoStandaloneAggregateRule run after ExtractCommonExpressionsRule
         // so that PushAggFunc can happen in fewer places.
         normalization.add(new PushAggFuncIntoStandaloneAggregateRule());
         normalization.add(new ListifyUnnestingFunctionRule());
-        normalization.add(new ConstantFoldingRule(appCtx));
         normalization.add(new RemoveRedundantSelectRule());
         normalization.add(new UnnestToDataScanRule());
         normalization.add(new MetaFunctionToMetaVariableRule());
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
index 56dc675..e64889d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
@@ -18,12 +18,15 @@
  */
 package org.apache.asterix.optimizer.rules;
 
-import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.IAObject;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.utils.ConstantExpressionUtil;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.IOptimizationContext;
-import org.apache.hyracks.algebricks.core.algebra.base.LogicalExpressionTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
@@ -89,50 +92,19 @@
         PhysicalOptimizationConfig physicalOptimizationConfig = context.getPhysicalOptimizationConfig();
         LimitOperator limitOp = (LimitOperator) opRef.getValue();
         OrderOperator orderOp = (OrderOperator) opRef2.getValue();
-        long topK = -1;
 
         // We don't push-down LIMIT into in-memory sort.
         if (orderOp.getPhysicalOperator().getOperatorTag() != PhysicalOperatorTag.STABLE_SORT) {
             return false;
         }
 
-        // Get the LIMIT constant
-        if (limitOp.getMaxObjects().getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-            // Currently, we support LIMIT with a constant value.
-            topK = AccessMethodUtils.getInt64Constant(limitOp.getMaxObjects());
-            // If topK is huge, there is no reason to use topK sort module
-            // since the original external sort's performance might be better.
-            if (topK > Integer.MAX_VALUE) {
-                return false;
-            }
-            if (topK < 0) {
-                topK = 0;
-            }
-        } else {
+        Integer topK = getOutputLimit(limitOp);
+        if (topK == null) {
             return false;
         }
 
-        // Get the offset constant if there is one. If one presents, then topK = topK + offset.
-        // This is because we can't apply offset to the external sort.
-        // Final topK will be applied through LIMIT.
-        if (limitOp.getOffset().getValue() != null) {
-            if (limitOp.getOffset().getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                long offset = AccessMethodUtils.getInt64Constant(limitOp.getOffset());
-                if (offset < 0) {
-                    offset = 0;
-                }
-                // Check the overflow case.
-                if (offset >= Integer.MAX_VALUE - topK) {
-                    return false;
-                }
-                topK += offset;
-            } else {
-                return false;
-            }
-        }
-
         // Create the new ORDER operator, set the topK value, and replace the current one.
-        OrderOperator newOrderOp = new OrderOperator(orderOp.getOrderExpressions(), (int) topK);
+        OrderOperator newOrderOp = new OrderOperator(orderOp.getOrderExpressions(), topK);
         newOrderOp.setSourceLocation(orderOp.getSourceLocation());
         newOrderOp.setPhysicalOperator(
                 new StableSortPOperator(physicalOptimizationConfig.getMaxFramesExternalSort(), newOrderOp.getTopK()));
@@ -146,4 +118,38 @@
         return true;
     }
 
+    static Integer getOutputLimit(LimitOperator limitOp) {
+        // Currently, we support LIMIT with a constant value.
+        ILogicalExpression maxObjectsExpr = limitOp.getMaxObjects().getValue();
+        IAObject maxObjectsValue = ConstantExpressionUtil.getConstantIaObject(maxObjectsExpr, ATypeTag.INTEGER);
+        if (maxObjectsValue == null) {
+            return null;
+        }
+        int topK = ((AInt32) maxObjectsValue).getIntegerValue();
+        if (topK < 0) {
+            topK = 0;
+        }
+
+        // Get the offset constant if there is one. If one presents, then topK = topK + offset.
+        // This is because we can't apply offset to the external sort.
+        // Final topK will be applied through LIMIT.
+        ILogicalExpression offsetExpr = limitOp.getOffset().getValue();
+        if (offsetExpr != null) {
+            IAObject offsetValue = ConstantExpressionUtil.getConstantIaObject(offsetExpr, ATypeTag.INTEGER);
+            if (offsetValue == null) {
+                return null;
+            }
+            int offset = ((AInt32) offsetValue).getIntegerValue();
+            if (offset < 0) {
+                offset = 0;
+            }
+            // Check the overflow case.
+            if (offset >= Integer.MAX_VALUE - topK) {
+                return null;
+            }
+            topK += offset;
+        }
+
+        return topK;
+    }
 }
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
index 921d231..5427ca6 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
@@ -25,7 +25,6 @@
 import org.apache.asterix.metadata.declared.DataSource;
 import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.rules.am.AccessMethodJobGenParams;
-import org.apache.asterix.optimizer.rules.am.AccessMethodUtils;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
@@ -67,7 +66,7 @@
         }
         context.addToDontApplySet(this, op);
 
-        Long outputLimit = getOutputLimit((LimitOperator) op);
+        Integer outputLimit = PushLimitIntoOrderByRule.getOutputLimit((LimitOperator) op);
         if (outputLimit == null) {
             // we cannot push if limit is not constant
             return false;
@@ -77,7 +76,7 @@
         if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.EXCHANGE) {
             childOp = childOp.getValue().getInputs().get(0);
         }
-        boolean changed = false;
+        boolean changed;
         if (childOp.getValue().getOperatorTag() == LogicalOperatorTag.SELECT) {
             changed = rewriteSelect(childOp, outputLimit);
         } else {
@@ -89,21 +88,7 @@
         return changed;
     }
 
-    private Long getOutputLimit(LimitOperator limit) {
-        if (limit.getMaxObjects().getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-            return null;
-        }
-        long outputLimit = AccessMethodUtils.getInt64Constant(limit.getMaxObjects());
-        if (limit.getOffset() != null && limit.getOffset().getValue() != null) {
-            if (limit.getOffset().getValue().getExpressionTag() != LogicalExpressionTag.CONSTANT) {
-                return null;
-            }
-            outputLimit += AccessMethodUtils.getInt64Constant(limit.getOffset());
-        }
-        return outputLimit;
-    }
-
-    private boolean rewriteSelect(Mutable<ILogicalOperator> op, long outputLimit) throws AlgebricksException {
+    private boolean rewriteSelect(Mutable<ILogicalOperator> op, int outputLimit) {
         SelectOperator select = (SelectOperator) op.getValue();
         Set<LogicalVariable> selectedVariables = new HashSet<>();
         select.getCondition().getValue().getUsedVariables(selectedVariables);
@@ -131,7 +116,7 @@
         return changed;
     }
 
-    private boolean setLimitForScanOrUnnestMap(ILogicalOperator op, long outputLimit) throws AlgebricksException {
+    private boolean setLimitForScanOrUnnestMap(ILogicalOperator op, int outputLimit) {
         if (op.getOperatorTag() == LogicalOperatorTag.DATASOURCESCAN) {
             DataSourceScanOperator scan = (DataSourceScanOperator) op;
             if (isScanPushable(scan, Collections.emptySet())) {
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
index 18a6597..5410a94 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
@@ -1085,38 +1085,27 @@
             ILogicalExpression e = p.first;
             // now look at the operator
             if (i < nOps) {
-                if (OperatorExpr.opIsComparison(ops.get(i))) {
-                    AbstractFunctionCallExpression c = createComparisonExpression(ops.get(i), sourceLoc);
+                OperatorType opType = ops.get(i);
+                boolean isCmpOp = OperatorExpr.opIsComparison(opType);
+                AbstractFunctionCallExpression f = createFunctionCallExpressionForBuiltinOperator(opType, sourceLoc);
 
-                    // chain the operators
-                    if (i == 0) {
-                        c.getArguments().add(new MutableObject<>(e));
-                        currExpr = c;
-                        if (op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.LEFT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
-                    } else {
-                        currExpr.getArguments().add(new MutableObject<>(e));
-                        c.getArguments().add(new MutableObject<>(currExpr));
-                        currExpr = c;
-                        if (i == 1 && op.isBroadcastOperand(i)) {
-                            BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
-                            bcast.setObject(BroadcastSide.RIGHT);
-                            c.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
-                        }
+                // chain the operators
+                if (i == 0) {
+                    f.getArguments().add(new MutableObject<>(e));
+                    currExpr = f;
+                    if (isCmpOp && op.isBroadcastOperand(i)) {
+                        BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                        bcast.setObject(BroadcastSide.LEFT);
+                        f.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                     }
                 } else {
-                    AbstractFunctionCallExpression f =
-                            createFunctionCallExpressionForBuiltinOperator(ops.get(i), sourceLoc);
-                    if (i == 0) {
-                        f.getArguments().add(new MutableObject<>(e));
-                        currExpr = f;
-                    } else {
-                        currExpr.getArguments().add(new MutableObject<>(e));
-                        f.getArguments().add(new MutableObject<>(currExpr));
-                        currExpr = f;
+                    currExpr.getArguments().add(new MutableObject<>(e));
+                    f.getArguments().add(new MutableObject<>(currExpr));
+                    currExpr = f;
+                    if (isCmpOp && i == 1 && op.isBroadcastOperand(i)) {
+                        BroadcastExpressionAnnotation bcast = new BroadcastExpressionAnnotation();
+                        bcast.setObject(BroadcastSide.RIGHT);
+                        f.getAnnotations().put(BroadcastExpressionAnnotation.BROADCAST_ANNOTATION_KEY, bcast);
                     }
                 }
             } else { // don't forget the last expression...
@@ -1359,53 +1348,52 @@
     public Pair<ILogicalOperator, LogicalVariable> visit(LimitClause lc, Mutable<ILogicalOperator> tupSource)
             throws CompilationException {
         SourceLocation sourceLoc = lc.getSourceLocation();
-        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = langExprToAlgExpression(lc.getLimitExpr(), tupSource);
         LimitOperator opLim;
+
+        Pair<ILogicalExpression, Mutable<ILogicalOperator>> p1 = langExprToAlgExpression(lc.getLimitExpr(), tupSource);
+        AbstractFunctionCallExpression maxObjectsExpr =
+                createFunctionCallExpression(BuiltinFunctions.TREAT_AS_INTEGER, lc.getLimitExpr().getSourceLocation());
+        maxObjectsExpr.getArguments().add(new MutableObject<>(p1.first));
+
         Expression offset = lc.getOffset();
         if (offset != null) {
             Pair<ILogicalExpression, Mutable<ILogicalOperator>> p2 = langExprToAlgExpression(offset, p1.second);
-            opLim = new LimitOperator(p1.first, p2.first);
+            AbstractFunctionCallExpression offsetExpr =
+                    createFunctionCallExpression(BuiltinFunctions.TREAT_AS_INTEGER, lc.getOffset().getSourceLocation());
+            offsetExpr.getArguments().add(new MutableObject<>(p2.first));
+            opLim = new LimitOperator(maxObjectsExpr, offsetExpr);
             opLim.getInputs().add(p2.second);
             opLim.setSourceLocation(sourceLoc);
         } else {
-            opLim = new LimitOperator(p1.first);
+            opLim = new LimitOperator(maxObjectsExpr);
             opLim.getInputs().add(p1.second);
             opLim.setSourceLocation(sourceLoc);
         }
         return new Pair<>(opLim, null);
     }
 
-    protected AbstractFunctionCallExpression createComparisonExpression(OperatorType t, SourceLocation sourceLoc) {
-        FunctionIdentifier fi = operatorTypeToFunctionIdentifier(t);
-        IFunctionInfo finfo = FunctionUtil.getFunctionInfo(fi);
-        ScalarFunctionCallExpression callExpr = new ScalarFunctionCallExpression(finfo);
-        callExpr.setSourceLocation(sourceLoc);
-        return callExpr;
-    }
-
-    private static FunctionIdentifier operatorTypeToFunctionIdentifier(OperatorType t) {
-        switch (t) {
-            case EQ:
-                return AlgebricksBuiltinFunctions.EQ;
-            case NEQ:
-                return AlgebricksBuiltinFunctions.NEQ;
-            case GT:
-                return AlgebricksBuiltinFunctions.GT;
-            case GE:
-                return AlgebricksBuiltinFunctions.GE;
-            case LT:
-                return AlgebricksBuiltinFunctions.LT;
-            case LE:
-                return AlgebricksBuiltinFunctions.LE;
-            default:
-                throw new IllegalStateException();
-        }
-    }
-
-    protected AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t,
+    private static AbstractFunctionCallExpression createFunctionCallExpressionForBuiltinOperator(OperatorType t,
             SourceLocation sourceLoc) throws CompilationException {
         FunctionIdentifier fid;
         switch (t) {
+            case EQ:
+                fid = AlgebricksBuiltinFunctions.EQ;
+                break;
+            case NEQ:
+                fid = AlgebricksBuiltinFunctions.NEQ;
+                break;
+            case GT:
+                fid = AlgebricksBuiltinFunctions.GT;
+                break;
+            case GE:
+                fid = AlgebricksBuiltinFunctions.GE;
+                break;
+            case LT:
+                fid = AlgebricksBuiltinFunctions.LT;
+                break;
+            case LE:
+                fid = AlgebricksBuiltinFunctions.LE;
+                break;
             case PLUS:
                 fid = AlgebricksBuiltinFunctions.NUMERIC_ADD;
                 break;
@@ -1440,6 +1428,11 @@
                 throw new CompilationException(ErrorCode.COMPILATION_ERROR, sourceLoc,
                         "Operator " + t + " is not yet implemented");
         }
+        return createFunctionCallExpression(fid, sourceLoc);
+    }
+
+    private static AbstractFunctionCallExpression createFunctionCallExpression(FunctionIdentifier fid,
+            SourceLocation sourceLoc) {
         ScalarFunctionCallExpression callExpr = new ScalarFunctionCallExpression(FunctionUtil.getFunctionInfo(fid));
         callExpr.setSourceLocation(sourceLoc);
         return callExpr;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
new file mode 100644
index 0000000..c8b2b7a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
@@ -0,0 +1,29 @@
+/*
+ * 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 expressions returning floating point numbers
+ *                 : in limit/offset
+ * Expected Result : Success
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 7.5/2.5 offset 5/2.5
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
new file mode 100644
index 0000000..ce55ac9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 5.5/2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
new file mode 100644
index 0000000..e479296
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 1 offset 3.5/2
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
new file mode 100644
index 0000000..b870b76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit "2"
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
new file mode 100644
index 0000000..3e3bfa7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 floating point type in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 2 offset true
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
new file mode 100644
index 0000000..d783bb1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 overflow error in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 9999999999
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
new file mode 100644
index 0000000..991f189
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
@@ -0,0 +1,28 @@
+/*
+ * 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 overflow error in limit/offset
+ * Expected Result : Failure
+ */
+
+select value t
+from [6,5,4,3,2,1] t
+order by t
+limit 1 offset 8888888888
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
new file mode 100644
index 0000000..5673992
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
@@ -0,0 +1,25 @@
+/*
+ * 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 test;
+
+explain select element c
+from  LineItem as c
+where (c.l_suppkey < 150)
+limit 7.5/1.5 offset 7.5/1.5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
new file mode 100644
index 0000000..fc9d42f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+/* scan and print a delimited text file */
+
+use test;
+
+explain
+  select element paper
+  from  DBLP1 as paper
+  limit 7.5/1.5 offset 7.5/1.5;
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
new file mode 100644
index 0000000..af64121
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
@@ -0,0 +1,3 @@
+3
+4
+5
\ No newline at end of file
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
new file mode 100644
index 0000000..d070b2b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
@@ -0,0 +1,34 @@
+distribute result [$$c]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$c])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$14(ASC), $$15(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              unnest-map [$$14, $$15, $$c] <- index-search("LineItem", 0, "test", "LineItem", FALSE, FALSE, 2, $$19, $$20, 2, $$19, $$20, TRUE, TRUE, TRUE) condition (lt($$c.getField(2), 150)) limit 10
+              -- BTREE_SEARCH  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  order (ASC, $$19) (ASC, $$20)
+                  -- STABLE_SORT [$$19(ASC), $$20(ASC)]  |PARTITIONED|
+                    exchange
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      project ([$$19, $$20])
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        exchange
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          unnest-map [$$18, $$19, $$20] <- index-search("idx_LineItem_suppkey", 0, "test", "LineItem", FALSE, FALSE, 0, 1, $$17, TRUE, FALSE, FALSE)
+                          -- BTREE_SEARCH  |PARTITIONED|
+                            exchange
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              assign [$$17] <- [150]
+                              -- ASSIGN  |PARTITIONED|
+                                empty-tuple-source
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
new file mode 100644
index 0000000..ee3e565
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
@@ -0,0 +1,20 @@
+distribute result [$$paper]
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  exchange
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    limit 5, 5
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      project ([$$paper])
+      -- STREAM_PROJECT  |PARTITIONED|
+        exchange
+        -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+          limit 10
+          -- STREAM_LIMIT  |PARTITIONED|
+            exchange
+            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+              data-scan []<-[$$12, $$paper] <- test.DBLP1 limit 10
+              -- DATASOURCE_SCAN  |PARTITIONED|
+                exchange
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  empty-tuple-source
+                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
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 2adba01..c5775d8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -10156,11 +10156,28 @@
   </test-group>
   <test-group name="limit">
     <test-case FilePath="limit">
+      <compilation-unit name="limit_type_01">
+        <output-dir compare="Text">limit_type_01</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
+      <compilation-unit name="limit_type_02">
+        <output-dir compare="Text">limit_type_01</output-dir>
+        <expected-error>ASX0039: Expected integer value, got 2.75 (in line 28, at column 10)</expected-error>
+        <expected-error>ASX0039: Expected integer value, got 1.75 (in line 28, at column 19)</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type string (in line 28, at column 7)</expected-error>
+        <expected-error>ASX1091: Type mismatch: expected value of type integer, but got the value of type boolean (in line 28, at column 16)</expected-error>
+        <expected-error>ASX0021: Source value 9999999999 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+        <expected-error>ASX0021: Source value 8888888888 is out of range that integer can hold - integer.MAX_VALUE: 2147483647, integer.MIN_VALUE: -2147483648</expected-error>
+        <source-location>false</source-location>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="limit">
       <compilation-unit name="push-limit-to-primary-scan">
         <output-dir compare="Text">push-limit-to-primary-scan</output-dir>
       </compilation-unit>
     </test-case>
-     <test-case FilePath="limit">
+    <test-case FilePath="limit">
        <compilation-unit name="push-limit-to-primary-scan-select">
          <output-dir compare="Text">push-limit-to-primary-scan-select</output-dir>
        </compilation-unit>
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 2fdbe1e..f570aa8 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
@@ -79,6 +79,7 @@
     public static final int CANNOT_COMPARE_COMPLEX = 36;
     public static final int TYPE_MISMATCH_GENERIC = 37;
     public static final int DIFFERENT_LIST_TYPE_ARGS = 38;
+    public static final int INTEGER_VALUE_EXPECTED = 39;
 
     public static final int UNSUPPORTED_JRE = 100;
 
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 ec18324..8c09d75 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -73,6 +73,7 @@
 35 = Unsupported multiple statements.
 36 = Cannot compare non-primitive values
 38 = Input contains different list types
+39 = Expected integer value, got %1$s
 
 100 = Unsupported JRE: %1$s
 
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
index c7441d6..8f24864 100644
--- a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
@@ -121,6 +121,7 @@
 import org.apache.asterix.om.typecomputer.impl.ToDoubleTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ToNumberTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.ToObjectTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.TreatAsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryBinaryInt64TypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryMinusTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.UnaryStringInt64TypeComputer;
@@ -1102,6 +1103,9 @@
     public static final FunctionIdentifier TO_STRING =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "to-string", 1);
 
+    public static final FunctionIdentifier TREAT_AS_INTEGER =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "treat-as-integer", 1);
+
     public static final FunctionIdentifier EXTERNAL_LOOKUP =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "external-lookup", FunctionIdentifier.VARARGS);
 
@@ -1297,6 +1301,8 @@
         addFunction(TO_OBJECT, ToObjectTypeComputer.INSTANCE, true);
         addFunction(TO_STRING, AStringTypeComputer.INSTANCE, true);
 
+        addPrivateFunction(TREAT_AS_INTEGER, TreatAsTypeComputer.INSTANCE_INTEGER, true);
+
         addFunction(IF_INF, IfNanOrInfTypeComputer.INSTANCE, true);
         addFunction(IF_MISSING, IfMissingTypeComputer.INSTANCE, true);
         addFunction(IF_MISSING_OR_NULL, IfMissingOrNullTypeComputer.INSTANCE, true);
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
new file mode 100644
index 0000000..1a5861b
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
@@ -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.
+ */
+
+package org.apache.asterix.om.typecomputer.impl;
+
+import org.apache.asterix.om.exceptions.TypeMismatchException;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+public class TreatAsTypeComputer extends AbstractResultTypeComputer {
+    public static final TreatAsTypeComputer INSTANCE_INTEGER = new TreatAsTypeComputer(BuiltinType.AINT32);
+
+    private final IAType type;
+
+    private TreatAsTypeComputer(IAType type) {
+        this.type = type;
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        IAType inputType = strippedInputTypes[0];
+        if (ATypeHierarchy.isCompatible(inputType.getTypeTag(), type.getTypeTag())) {
+            return type;
+        } else {
+            throw new TypeMismatchException(expr.getSourceLocation(), inputType.getTypeTag(), type.getTypeTag());
+        }
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
new file mode 100644
index 0000000..2c3e148
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.asterix.runtime.evaluators.functions;
+
+import java.io.DataOutput;
+
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.common.exceptions.RuntimeDataException;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.om.base.AInt32;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.asterix.om.types.hierachy.ATypeHierarchy;
+import org.apache.asterix.runtime.evaluators.base.AbstractScalarFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.exceptions.TypeMismatchException;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.api.IPointable;
+import org.apache.hyracks.data.std.primitive.VoidPointable;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class TreatAsIntegerDescriptor extends AbstractScalarFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new TreatAsIntegerDescriptor();
+        }
+    };
+
+    @Override
+    public IScalarEvaluatorFactory createEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IScalarEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IScalarEvaluator createScalarEvaluator(final IHyracksTaskContext ctx) throws HyracksDataException {
+
+                final IScalarEvaluator inputEval = args[0].createScalarEvaluator(ctx);
+                final IPointable inputArg = new VoidPointable();
+                final ArrayBackedValueStorage resultStorage = new ArrayBackedValueStorage();
+                final DataOutput out = resultStorage.getDataOutput();
+                final AMutableInt32 aInt32 = new AMutableInt32(0);
+
+                @SuppressWarnings("unchecked")
+                final ISerializerDeserializer<AInt32> int32Ser =
+                        SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+
+                return new IScalarEvaluator() {
+                    @Override
+                    public void evaluate(IFrameTupleReference tuple, IPointable result) throws HyracksDataException {
+                        inputEval.evaluate(tuple, inputArg);
+
+                        int intValue;
+                        byte[] bytes = inputArg.getByteArray();
+                        int startOffset = inputArg.getStartOffset();
+                        ATypeTag tt = ATypeTag.VALUE_TYPE_MAPPING[bytes[startOffset]];
+                        switch (tt) {
+                            case TINYINT:
+                            case SMALLINT:
+                            case INTEGER:
+                            case BIGINT:
+                                intValue = ATypeHierarchy.getIntegerValue(getIdentifier().getName(), 0, bytes,
+                                        startOffset, true);
+                                break;
+                            case FLOAT:
+                            case DOUBLE:
+                                double doubleValue =
+                                        ATypeHierarchy.getDoubleValue(getIdentifier().getName(), 0, bytes, startOffset);
+                                intValue = asInt(doubleValue);
+                                break;
+                            default:
+                                throw new TypeMismatchException(sourceLoc, bytes[startOffset],
+                                        ATypeTag.SERIALIZED_INT8_TYPE_TAG, ATypeTag.SERIALIZED_INT16_TYPE_TAG,
+                                        ATypeTag.SERIALIZED_INT32_TYPE_TAG, ATypeTag.SERIALIZED_INT64_TYPE_TAG,
+                                        ATypeTag.SERIALIZED_FLOAT_TYPE_TAG, ATypeTag.SERIALIZED_DOUBLE_TYPE_TAG);
+                        }
+
+                        resultStorage.reset();
+                        aInt32.setValue(intValue);
+                        int32Ser.serialize(aInt32, out);
+                        result.set(resultStorage);
+                    }
+
+                    private int asInt(double d) throws HyracksDataException {
+                        if (Double.isFinite(d)) {
+                            long v = (long) d;
+                            if (v == d && Integer.MIN_VALUE <= v && v <= Integer.MAX_VALUE) {
+                                return (int) v;
+                            }
+                        }
+                        throw new RuntimeDataException(ErrorCode.INTEGER_VALUE_EXPECTED, sourceLoc, d);
+                    }
+                };
+            }
+        };
+
+    }
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.TREAT_AS_INTEGER;
+    }
+}
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
index 25d90d3..04f4a1e 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
@@ -281,6 +281,7 @@
 import org.apache.asterix.runtime.evaluators.functions.ToNumberDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ToObjectDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.ToStringDescriptor;
+import org.apache.asterix.runtime.evaluators.functions.TreatAsIntegerDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.UUIDDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.binary.BinaryConcatDescriptor;
 import org.apache.asterix.runtime.evaluators.functions.binary.BinaryLengthDescriptor;
@@ -790,6 +791,8 @@
         fc.addGenerated(ToObjectDescriptor.FACTORY);
         fc.addGenerated(ToStringDescriptor.FACTORY);
 
+        fc.addGenerated(TreatAsIntegerDescriptor.FACTORY);
+
         // Cast function
         fc.addGenerated(CastTypeDescriptor.FACTORY);
         fc.addGenerated(CastTypeLaxDescriptor.FACTORY);

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

Gerrit-MessageType: merged
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 6
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7966/ (3/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/287/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2018/ (3/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/4452/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3993/ (11/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/Fi9G3U : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-asterix-app/4475/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/313/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4030/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Taewoo Kim, Till Westmann, Jenkins,

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

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

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

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................

[ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

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

Details:
- Fixed NPE in optimizer when limit/offset is non-integer
- Allow float/double values that are integers in limit/offset
- Make consistent limit/offset value types (integer) between
  runtime and optimizer rules
- Add internal function treat-as-integer() that returns integer
  if input number conforms to the integer type or fails otherwise

Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
23 files changed, 613 insertions(+), 117 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/88/2788/5
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2788
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Analytics Compatibility Compilation Successful
https://goo.gl/sZq7BT : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4032/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3917/ (3/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3943/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3: Contrib+1

BAD Compatibility Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/311/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Analytics Compatibility Compilation Successful
https://goo.gl/GZWme5 : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/289/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4034/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Jenkins,

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

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

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

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................

[ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

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

Details:
- Fixed NPE in optimizer when limit/offset is non-integer
- Allow float/double values that are integers in limit/offset
- Make consistent limit/offset value types (integer) between
  runtime and optimizer rules
- Add internal function treat-as-integer() that returns integer
  if input number conforms to the integer type or fails otherwise

Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
23 files changed, 613 insertions(+), 117 deletions(-)


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

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7942/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4250/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4573/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/266/ (11/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-sql-execution/4006/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9500/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4229/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9478/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7968/ (3/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3920/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3941/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Analytics Compatibility Compilation Successful
https://goo.gl/oZBaaq : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/99QEhC : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2039/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/rLmfQa : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/9Y5DAh : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4577/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/289/ (13/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3995/ (9/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage/4551/ (2/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

(1 comment)

https://asterix-gerrit.ics.uci.edu/#/c/2788/4/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
File asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml:

PS4, Line 10166: ASX0038
Remove the error code from the expected result?


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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: Yes

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-asterix-app-java10/287/ (10/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7970/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-sonar/7945/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/Nr12QC : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2: Contrib+1

Analytics Compatibility Tests Successful
https://goo.gl/ZvnKru : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-verify-storage-jre10/284/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-verify-no-installer-app/4423/ (12/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Analytics Compatibility Compilation Successful
https://goo.gl/8TfJkX : SUCCESS

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-ensure-ancestor/2015/ (5/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4252/ (6/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Till Westmann (Code Review)" <do...@asterixdb.incubator.apache.org>.
Till Westmann has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Dmitry Lychagin (Code Review)" <do...@asterixdb.incubator.apache.org>.
Hello Anon. E. Moose #1000171, Taewoo Kim, Till Westmann, Jenkins,

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

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

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

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................

[ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

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

Details:
- Fixed NPE in optimizer when limit/offset is non-integer
- Allow float/double values that are integers in limit/offset
- Make consistent limit/offset value types (integer) between
  runtime and optimizer rules
- Add internal function treat-as-integer() that returns integer
  if input number conforms to the integer type or fails otherwise

Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
---
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoPrimarySearchRule.java
M asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_01/limit_type_01.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.1.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.2.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.3.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.4.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/limit_type_02/limit_type_02.6.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.query.sqlpp
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/limit_type_01/limit_type_01.1.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-lookup/push-limit-to-primary-lookup.5.adm
A asterixdb/asterix-app/src/test/resources/runtimets/results/limit/push-limit-to-primary-scan/push-limit-to-primary-scan.5.adm
M asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
M asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
M asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
M asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
A asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/TreatAsTypeComputer.java
A asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/evaluators/functions/TreatAsIntegerDescriptor.java
M asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
23 files changed, 613 insertions(+), 117 deletions(-)


  git pull ssh://asterix-gerrit.ics.uci.edu:29418/asterixdb refs/changes/88/2788/4
-- 
To view, visit https://asterix-gerrit.ics.uci.edu/2788
To unsubscribe, visit https://asterix-gerrit.ics.uci.edu/settings

Gerrit-MessageType: newpatchset
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

Build Started https://asterix-jenkins.ics.uci.edu/job/hyracks-gerrit/3945/ (7/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-assemblies/4226/ (8/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3: Integration-Tests+1

Integration Tests Successful

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-source-format/3970/ (1/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 5:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 5
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 2:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 2
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Taewoo Kim (Code Review)" <do...@asterixdb.incubator.apache.org>.
Taewoo Kim has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3: Code-Review+2

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 4:

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

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 4
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 3: Contrib-2

BAD Compatibility Tests Failed

https://asterix-jenkins.ics.uci.edu/job/asterixbad-compat/3421/ : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 3
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-Reviewer: Luo Chen <cl...@uci.edu>
Gerrit-Reviewer: Taewoo Kim <wa...@gmail.com>
Gerrit-Reviewer: Till Westmann <ti...@apache.org>
Gerrit-Reviewer: Xikui Wang <xk...@gmail.com>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Jenkins (Code Review)" <do...@asterixdb.incubator.apache.org>.
Jenkins has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1:

Build Started https://asterix-jenkins.ics.uci.edu/job/asterix-gerrit-notopic/9475/ (4/13)

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No

Change in asterixdb[master]: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause

Posted by "Anon. E. Moose (Code Review)" <do...@asterixdb.incubator.apache.org>.
Anon. E. Moose #1000171 has posted comments on this change.

Change subject: [ASTERIXDB-2417][COMP] Fix issues with LIMIT clause
......................................................................


Patch Set 1: Contrib-2

Analytics Compatibility Tests Failed
https://goo.gl/YhnD4c : UNSTABLE

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

Gerrit-MessageType: comment
Gerrit-Change-Id: I986fab3e79b072aa2441af293f3e16c3f37bf508
Gerrit-PatchSet: 1
Gerrit-Project: asterixdb
Gerrit-Branch: master
Gerrit-Owner: Dmitry Lychagin <dm...@couchbase.com>
Gerrit-Reviewer: Anon. E. Moose #1000171
Gerrit-Reviewer: Jenkins <je...@fulliautomatix.ics.uci.edu>
Gerrit-HasComments: No