You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2016/10/04 17:06:56 UTC

asterixdb git commit: ASTERIXDB-1671: fix non-positive limit with order by.

Repository: asterixdb
Updated Branches:
  refs/heads/master 2e3d16739 -> 6adeec2d1


ASTERIXDB-1671: fix non-positive limit with order by.

Change-Id: Id28218c100e4cca0f94e37eda82d3bc6ae78322f
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1240
Sonar-Qube: Jenkins <je...@fulliautomatix.ics.uci.edu>
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Taewoo Kim <wa...@yahoo.com>


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

Branch: refs/heads/master
Commit: 6adeec2d19f0051d5696f9453e52b86e7a02461d
Parents: 2e3d167
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Mon Oct 3 21:21:31 2016 -0700
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Tue Oct 4 10:06:26 2016 -0700

----------------------------------------------------------------------
 .../rules/PushLimitIntoOrderByRule.java         | 16 +++++--
 .../queries/query-ASTERIXDB-1671.sqlpp          | 40 +++++++++++++++++
 .../results/query-ASTERIXDB-1671.plan           | 17 +++++++
 .../query-ASTERIXDB-1671-2.1.ddl.sqlpp          | 47 ++++++++++++++++++++
 .../query-ASTERIXDB-1671-2.2.update.sqlpp       | 24 ++++++++++
 .../query-ASTERIXDB-1671-2.3.query.sqlpp        | 27 +++++++++++
 .../query-ASTERIXDB-1671.1.ddl.sqlpp            | 47 ++++++++++++++++++++
 .../query-ASTERIXDB-1671.2.update.sqlpp         | 24 ++++++++++
 .../query-ASTERIXDB-1671.3.query.sqlpp          | 27 +++++++++++
 .../query-ASTERIXDB-1671.1.adm                  |  0
 .../resources/runtimets/testsuite_sqlpp.xml     | 10 +++++
 .../std/sort/HybridTopKSortRunGenerator.java    |  3 ++
 12 files changed, 278 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/PushLimitIntoOrderByRule.java
----------------------------------------------------------------------
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 b573ae4..4c0247f 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
@@ -96,8 +96,6 @@ public class PushLimitIntoOrderByRule implements IAlgebraicRewriteRule {
             return false;
         }
 
-        boolean needToCheckOffsetValue = true;
-
         // Get the LIMIT constant
         if (limitOp.getMaxObjects().getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
             // Currently, we support LIMIT with a constant value.
@@ -107,6 +105,9 @@ public class PushLimitIntoOrderByRule implements IAlgebraicRewriteRule {
             if (topK > Integer.MAX_VALUE) {
                 return false;
             }
+            if (topK < 0) {
+                topK = 0;
+            }
         } else {
             return false;
         }
@@ -116,7 +117,15 @@ public class PushLimitIntoOrderByRule implements IAlgebraicRewriteRule {
         // Final topK will be applied through LIMIT.
         if (limitOp.getOffset().getValue() != null) {
             if (limitOp.getOffset().getValue().getExpressionTag() == LogicalExpressionTag.CONSTANT) {
-                topK = topK + ((int) AccessMethodUtils.getInt64Constant(limitOp.getOffset()));
+                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;
             }
@@ -133,7 +142,6 @@ public class PushLimitIntoOrderByRule implements IAlgebraicRewriteRule {
         opRef2.setValue(newOrderOp);
         context.computeAndSetTypeEnvironmentForOperator(newOrderOp);
         context.addToDontApplySet(this, limitOp);
-
         return true;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1671.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1671.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1671.sqlpp
new file mode 100644
index 0000000..33b40b2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1671.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+DROP DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+
+USE tpch;
+
+
+CREATE TYPE LineItemType AS {
+  l_orderkey : bigint,
+  l_linenumber : bigint
+}
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_orderkey,l_linenumber;
+
+-- Tests that the query does not do a complete sort over the data.
+
+SELECT  *
+FROM  LineItem l
+ORDER BY l.l_returnflag, l.l_linestatus
+LIMIT -1;
+
+DROP DATAVERSE tpch;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1671.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1671.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1671.plan
new file mode 100644
index 0000000..6b370af
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1671.plan
@@ -0,0 +1,17 @@
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_LIMIT  |UNPARTITIONED|
+      -- STREAM_PROJECT  |PARTITIONED|
+        -- ASSIGN  |PARTITIONED|
+          -- STREAM_PROJECT  |PARTITIONED|
+            -- SORT_MERGE_EXCHANGE [$$11(ASC), $$12(ASC) ]  |PARTITIONED|
+              -- STREAM_LIMIT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- STABLE_SORT [topK: 0] [$$11(ASC), $$12(ASC)]  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- ASSIGN  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671-2/query-ASTERIXDB-1671-2.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671-2/query-ASTERIXDB-1671-2.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671-2/query-ASTERIXDB-1671-2.2.update.sqlpp
new file mode 100644
index 0000000..5fe734c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671-2/query-ASTERIXDB-1671-2.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use tpch;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671-2/query-ASTERIXDB-1671-2.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671-2/query-ASTERIXDB-1671-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671-2/query-ASTERIXDB-1671-2.3.query.sqlpp
new file mode 100644
index 0000000..51c4ceb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671-2/query-ASTERIXDB-1671-2.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+-- Tests that the query with LIMIT -1 can be executed correctly.
+
+SELECT  *
+FROM  LineItem l
+ORDER BY l.l_returnflag, l.l_linestatus
+LIMIT -1;

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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.2.update.sqlpp
new file mode 100644
index 0000000..5fe734c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.2.update.sqlpp
@@ -0,0 +1,24 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+use tpch;
+
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.3.query.sqlpp
new file mode 100644
index 0000000..5a51067
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.3.query.sqlpp
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+USE tpch;
+
+-- Tests that the query with LIMIT 0 can be executed correctly.
+
+SELECT  *
+FROM  LineItem l
+ORDER BY l.l_returnflag, l.l_linestatus
+LIMIT 0;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/query-ASTERIXDB-1671/query-ASTERIXDB-1671.1.adm
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 5dd0bd0..a17aa5b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -3145,6 +3145,16 @@
       </compilation-unit>
     </test-case>
     <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1671">
+        <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="query-ASTERIXDB-1671-2">
+        <output-dir compare="Text">query-ASTERIXDB-1671</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
       <compilation-unit name="uuid">
         <output-dir compare="Text">uuid</output-dir>
       </compilation-unit>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/6adeec2d/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
index 9be4bc6..5591ef7 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/HybridTopKSortRunGenerator.java
@@ -67,6 +67,9 @@ public class HybridTopKSortRunGenerator extends HeapSortRunGenerator {
 
     @Override
     public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+        if (topK <= 0) {
+            return;
+        }
         inAccessor.reset(buffer);
         if (tupleSorter != null) {
             boolean isBadK = false;