korlov42 commented on code in PR #5479:
URL: https://github.com/apache/ignite-3/pull/5479#discussion_r2020738667


##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/LogicalRelImplementor.java:
##########
@@ -1168,6 +1154,21 @@ private static boolean canFuseProjectionInto(RelNode 
rel) {
         return joinProjection;
     }
 
+    private long validateAndGetFetchOffsetParams(RexNode node, String op) {
+        SqlScalar<RowT, Number> sqlScalar = expressionFactory.scalar(node);
+        Number offset = sqlScalar.get(ctx);
+
+        if (offset instanceof BigDecimal) {
+            BigDecimal offset0 = (BigDecimal) offset;
+            if (offset0.signum() == -1 || offset0.compareTo(LIMIT_UPPER) > 0) {
+                throw new SqlException(Sql.STMT_VALIDATION_ERR,
+                        "Illegal value of " + op + ". The value must be 
positive and less than (" + Long.MAX_VALUE + ")");

Review Comment:
   let's reuse `IgniteResource`



##########
modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/DynamicParametersTest.java:
##########
@@ -724,12 +724,12 @@ public Stream<DynamicTest> testLimitOffset() {
 
                 checkStatement(setup)
                         .sql("SELECT * FROM t1 LIMIT ?", Unspecified.UNKNOWN)
-                        .parameterTypes(nullable(NativeTypes.INT32))
+                        .parameterTypes(nullable(NativeTypes.INT64))
                         .ok(),
 
                 checkStatement(setup)
                         .sql("SELECT * FROM t1 LIMIT ?", "a")
-                        .fails("Incorrect type of a dynamic parameter. 
Expected <INTEGER> but got <VARCHAR"),
+                        .fails("Incorrect type of a dynamic parameter. 
Expected <DECIMAL> but got <VARCHAR>"),

Review Comment:
   doesn't this look strange to you? you derived type of the param as INT64 
(aka `BIGINT`), but the very next case reports that `DECIMAL` was expected



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/LimitNode.java:
##########
@@ -69,16 +79,17 @@ public void request(int rowsCnt) throws Exception {
             return;
         }
 
-        if (offset > 0 && rowsProcessed == 0) {
-            rowsCnt = offset + rowsCnt;
-        }
-
-        waiting = rowsCnt;
+        requested = rowsCnt;

Review Comment:
   let's add assertion that previous requested count is not overridden 



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/rel/LimitNode.java:
##########
@@ -19,45 +19,55 @@
 
 import static org.apache.ignite.internal.util.CollectionUtils.nullOrEmpty;
 
-import java.util.function.Supplier;
 import org.apache.ignite.internal.sql.engine.exec.ExecutionContext;
-import org.jetbrains.annotations.Nullable;
+import org.apache.ignite.internal.sql.engine.util.IgniteMath;
 
 /** Offset, fetch|limit support node. */
 public class LimitNode<RowT> extends AbstractNode<RowT> implements 
SingleNode<RowT>, Downstream<RowT> {
-    /** Offset if its present, otherwise 0. */
-    private final int offset;
+    /** Offset param. */
+    private final long offset;
 
-    /** Fetch if its present, otherwise 0. */
-    private final int fetch;
+    /** Fetch param. */
+    private final long fetch;
 
-    /** Already processed (pushed to upstream) rows count. */
-    private int rowsProcessed;
+    /** Fetch can be unset. */
+    private final boolean fetchUndefined;
 
-    /** Fetch can be unset, in this case we need all rows. */
-    private @Nullable Supplier<Integer> fetchNode;
+    /** Already processed (pushed to upstream) rows count. */
+    private long rowsProcessed;
 
     /** Waiting results counter. */
     private int waiting;
 
+    /** Upper requested rows. */
+    private int requested;
+
     /**
      * Constructor.
      *
      * @param ctx     Execution context.
      */
     public LimitNode(
             ExecutionContext<RowT> ctx,
-            Supplier<Integer> offsetNode,
-            Supplier<Integer> fetchNode
+            long offset,
+            long fetch
     ) {
         super(ctx);
 
-        offset = offsetNode == null ? 0 : offsetNode.get();
-        fetch = fetchNode == null ? 0 : fetchNode.get();
-        this.fetchNode = fetchNode;
+        this.offset = offset;
+        fetchUndefined = fetch == -1;
+        this.fetch = fetch == -1 ? 0 : fetch;
     }
 
-    /** {@inheritDoc} */
+    /**
+     * Several cases are need to be processed.

Review Comment:
   this is definitely not a javadoc.



##########
modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/util/IgniteResource.java:
##########
@@ -49,8 +49,8 @@ public interface IgniteResource {
     @BaseMessage("Illegal aggregate function. {0} is unsupported at the 
moment")
     ExInst<SqlValidatorException> unsupportedAggregationFunction(String a0);
 
-    @BaseMessage("Illegal value of {0}. The value must be positive and less 
than Integer.MAX_VALUE (" + Integer.MAX_VALUE + ")")
-    ExInst<SqlValidatorException> correctIntegerLimit(String a0);
+    @BaseMessage("Illegal value of {0}. The value must be positive and less 
than (" + Long.MAX_VALUE + ")")

Review Comment:
   ```suggestion
       @BaseMessage("Illegal value of {0}. The value must be positive and less 
than " + Long.MAX_VALUE)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscr...@ignite.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to