You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2021/02/11 01:03:35 UTC

[impala] branch master updated: IMPALA-9979: part 2: partitioned top-n

This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git


The following commit(s) were added to refs/heads/master by this push:
     new b42c649  IMPALA-9979: part 2: partitioned top-n
b42c649 is described below

commit b42c64993d46893488a667fb9c425548fdf964ab
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Tue Feb 2 14:02:12 2021 -0800

    IMPALA-9979: part 2: partitioned top-n
    
    Planner changes:
    ---------------
    The planner now identifies predicates that can be converted into
    limits in a partitioned or unpartitioned top-n with the following
    method:
    * Push down predicates that reference analytic tuple into inline view.
      These will be evaluated after the analytic plan for the inline
      SelectStmt is generated.
    * Identify predicates that reference the analytic tuple and could
      be converted to limits.
    * If they can be applied to the last sort group of the analytic
      plan, and the windows are all compatible, then the lowest
      limit gets converted into a limit in the top N.
    * Otherwise generate a select node with the conjuncts. We add
      logic to merge SELECT nodes to avoid generating duplicates
      from inside and outside the inline view.
    * The pushed predicate is still added to the SELECT node
      because it is necessary for correctness for predicates
      like '=' to filter additional rows and also the limit
      pushdown optimization looks for analytic predicates
      there, so retaining all predicates simplifies that.
      The selectivity of the predicate is adjusted so that
      cardinality estimates remain accurate.
    
    The optimization can be disabled by setting
    ANALYTIC_RANK_PUSHDOWN_THRESHOLD=0. By default it is
    only enabled for limits of 1000 or less, because the
    in-memory Top-N may perform significantly worse than
    a full sort for large heaps (since updating the heap
    for every input row ends up being more expensive than
    doing a traditional sort). We could probably optimize
    this more with better tuning so that it can gracefully
    fall back to doing the full sort at runtime.
    
    rank() and row_number() are handled. rank() needs support in
    the TopN node to include ties for the last place, which is
    also added in this patch.
    
    If predicates are trivially false, we generate empty nodes.
    
    This interacts with the limit pushdwon optimization. The limit
    pushdown optimization is applied after the partitioned top-n
    is generated, and can sometimes result in more optimal plans,
    so it is generalized to handle pushing into partitioned top-n
    nodes.
    
    Backend changes:
    ---------------
    The top-n node in the backend is augmented to handle
    the partitioned case, for which we use a std::map and a
    comparator based on the partition exprs. The partitioned
    top-n node has a soft limit of 64MB on the size of the
    in-memory heaps and can spill with use of an embedded Sorter.
    The current implementation tries to evict heaps that are
    less effective at filtering rows.
    
    Limitations:
    -----------
    There are several possible extensions to this that we did not do:
    * dense_rank() is not supported because it would require additional
      backend support - IMPALA-10014.
    * ntile() is not supported because it would require additional
      backend support - IMPALA-10174.
    * Only one predicate per analytic is pushed.
    * Redundant rank()/row_number() predicates are not merged,
      only the lowest is chosen.
    * Lower bounds are not converted into OFFSET.
    * The analytic operator cannot be eliminated even if the analytic
      expression was only used in the predicate.
    * This doesn't push predicates into UNION - IMPALA-10013
    * Always false predicates don't result in empty plan - IMPALA-10015
    
    Tests:
    -----
    * Planner tests - added tests that exercise the interesting code
      paths added in planning.
      - Predicate ordering in SELECT nodes changed in a couple of cases
        because some predicates were pushed into the inline views.
    * Modified SORT targeted perf tests to avoid conversion to Top-N
    * Added targeted perf test for partitioned top-n.
    * End-to-end tests
     - Unpartitioned Top-N end-to-end tests
     - Basic partitioning and duplicate handling tests on functional
     - Similar basic tests on larger inputs from TPC-DS and with
       larger partition counts.
     - I inspected the results and also ran the same tests with
       analytic_rank_pushdown_threshold=0 to confirm that the
       results were the same as with the full sort.
     - Fallback to spilling sort.
    
    Perf:
    -----
    Added a targeted benchmark that goes from ~2s to ~1s with
    mt_dop=8 on TPC-H 30 on my desktop.
    
    Change-Id: Ic638af9495981d889a4cb7455a71e8be0eb1a8e5
    Reviewed-on: http://gerrit.cloudera.org:8080/16242
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/codegen/gen_ir_descriptions.py              |    6 +-
 be/src/exec/exec-node.cc                           |    3 +-
 be/src/exec/topn-node-ir.cc                        |   83 +-
 be/src/exec/topn-node.cc                           |  466 +++++-
 be/src/exec/topn-node.h                            |  236 ++-
 be/src/exprs/slot-ref.h                            |    6 +-
 be/src/service/query-options-test.cc               |    1 +
 be/src/service/query-options.cc                    |   14 +
 be/src/service/query-options.h                     |    4 +-
 be/src/util/priority-queue.h                       |   12 +-
 be/src/util/tuple-row-compare.h                    |   24 +
 common/thrift/ImpalaInternalService.thrift         |    3 +
 common/thrift/ImpalaService.thrift                 |    9 +
 common/thrift/PlanNodes.thrift                     |   15 +-
 .../main/java/org/apache/impala/analysis/Expr.java |   24 +
 .../java/org/apache/impala/analysis/SlotRef.java   |    9 +-
 .../java/org/apache/impala/analysis/SortInfo.java  |   12 +-
 .../apache/impala/planner/AnalyticEvalNode.java    |   96 +-
 .../org/apache/impala/planner/AnalyticPlanner.java |  236 ++-
 .../apache/impala/planner/DistributedPlanner.java  |   43 +-
 .../java/org/apache/impala/planner/PlanNode.java   |   40 +
 .../java/org/apache/impala/planner/SelectNode.java |   30 +-
 .../apache/impala/planner/SingleNodePlanner.java   |  110 +-
 .../java/org/apache/impala/planner/SortNode.java   |  170 +-
 .../org/apache/impala/planner/CardinalityTest.java |   40 +
 .../org/apache/impala/planner/PlannerTest.java     |   29 +-
 .../queries/PlannerTest/analytic-fns.test          |    4 +-
 .../PlannerTest/analytic-rank-pushdown.test        | 1678 ++++++++++++++++++++
 .../limit-pushdown-partitioned-top-n.test          |  856 ++++++++++
 .../queries/PlannerTest/resource-requirements.test |  201 ++-
 .../queries/PlannerTest/tpcds/tpcds-q44.test       |  332 ++--
 .../queries/PlannerTest/tpcds/tpcds-q67.test       |    2 +
 .../queries/PlannerTest/tpcds/tpcds-q70.test       |   30 +-
 .../analytic-fns-tpcds-partitioned-topn.test       |  325 ++++
 .../queries/QueryTest/partitioned-top-n.test       |   82 +
 .../queries/QueryTest/spilling.test                |   66 +
 .../functional-query/queries/QueryTest/top-n.test  |   89 ++
 .../queries/primitive_orderby_all.test             |    2 +-
 .../queries/primitive_orderby_bigint.test          |    2 +-
 .../primitive_orderby_bigint_expression.test       |    2 +-
 .../queries/primitive_top-n_partitioned.test       |   45 +
 tests/experiments/test_targeted_perf.py            |    3 +
 tests/query_test/test_analytic_tpcds.py            |    5 +
 tests/query_test/test_queries.py                   |    5 +
 44 files changed, 5065 insertions(+), 385 deletions(-)

diff --git a/be/src/codegen/gen_ir_descriptions.py b/be/src/codegen/gen_ir_descriptions.py
index 6033470..e2459bf 100755
--- a/be/src/codegen/gen_ir_descriptions.py
+++ b/be/src/codegen/gen_ir_descriptions.py
@@ -187,8 +187,10 @@ ir_functions = [
    "_ZN6impala8RawValue20GetHashValueFastHashEPKvRKNS_10ColumnTypeEm"],
   ["RAW_VALUE_GET_HASH_VALUE_FAST_HASH32",
    "_ZN6impala8RawValue22GetHashValueFastHash32EPKvRKNS_10ColumnTypeEj"],
-  ["TOPN_NODE_INSERT_BATCH",
-   "_ZN6impala8TopNNode11InsertBatchEPNS_8RowBatchE"],
+  ["TOPN_NODE_INSERT_BATCH_UNPARTITIONED",
+   "_ZN6impala8TopNNode24InsertBatchUnpartitionedEPNS_12RuntimeStateEPNS_8RowBatchE"],
+  ["TOPN_NODE_INSERT_BATCH_PARTITIONED",
+   "_ZN6impala8TopNNode22InsertBatchPartitionedEPNS_12RuntimeStateEPNS_8RowBatchE"],
   ["MEMPOOL_ALLOCATE",
    "_ZN6impala7MemPool8AllocateILb0EEEPhli"],
   ["MEMPOOL_CHECKED_ALLOCATE",
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index d4a7d91..26ac423 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -184,7 +184,8 @@ Status PlanNode::CreatePlanNode(
     case TPlanNodeType::SORT_NODE:
       if (tnode.sort_node.type == TSortType::PARTIAL) {
         *node = pool->Add(new PartialSortPlanNode());
-      } else if (tnode.sort_node.type == TSortType::TOPN) {
+      } else if (tnode.sort_node.type == TSortType::TOPN ||
+          tnode.sort_node.type == TSortType::PARTITIONED_TOPN) {
         *node = pool->Add(new TopNPlanNode());
       } else {
         DCHECK(tnode.sort_node.type == TSortType::TOTAL);
diff --git a/be/src/exec/topn-node-ir.cc b/be/src/exec/topn-node-ir.cc
index d64b2f0..f173c38 100644
--- a/be/src/exec/topn-node-ir.cc
+++ b/be/src/exec/topn-node-ir.cc
@@ -17,9 +17,15 @@
 
 #include "exec/topn-node.h"
 
+#include "common/compiler-util.h"
+#include "util/debug-util.h"
+
+#include "common/names.h"
+
 using namespace impala;
 
-void TopNNode::InsertBatch(RowBatch* batch) {
+void TopNNode::InsertBatchUnpartitioned(RuntimeState* state, RowBatch* batch) {
+  DCHECK(!is_partitioned());
   // TODO: after inlining the comparator calls with codegen - IMPALA-4065 - we could
   // probably squeeze more performance out of this loop by ensure that as many loads
   // are hoisted out of the loop as possible (either via code changes or __restrict__)
@@ -52,9 +58,9 @@ int TopNNode::Heap::InsertTupleRow(TopNNode* node, TupleRow* input_row) {
   node->tmp_tuple_->MaterializeExprs<false, true>(input_row, tuple_desc,
       node->output_tuple_expr_evals_, nullptr);
   if (include_ties()) {
-    return InsertTupleWithTieHandling(node, node->tmp_tuple_);
+    return InsertTupleWithTieHandling(*node->order_cmp_, node, node->tmp_tuple_);
   } else {
-    if (node->tuple_row_less_than_->Less(node->tmp_tuple_, top_tuple)) {
+    if (node->order_cmp_->Less(node->tmp_tuple_, top_tuple)) {
       // Pop off the old head, and replace with the new tuple. Deep copy into 'top_tuple'
       // to reuse the fixed-length memory of 'top_tuple'.
       node->tmp_tuple_->DeepCopy(top_tuple, tuple_desc, node->tuple_pool_.get());
@@ -67,7 +73,7 @@ int TopNNode::Heap::InsertTupleRow(TopNNode* node, TupleRow* input_row) {
 }
 
 int TopNNode::Heap::InsertTupleWithTieHandling(
-    TopNNode* node, Tuple* materialized_tuple) {
+    const TupleRowComparator& cmp, TopNNode* node, Tuple* materialized_tuple) {
   DCHECK(include_ties());
   DCHECK_EQ(capacity_, priority_queue_.Size())
         << "Ties only need special handling when heap is at capacity";
@@ -76,8 +82,8 @@ int TopNNode::Heap::InsertTupleWithTieHandling(
   // If we need to retain ties with the current head, the logic is more complex - we
   // have a logical heap in indices [0, heap_capacity()) of 'priority_queue_' plus
   // some number of tuples in 'overflowed_ties_' that are equal to priority_queue_.Top()
-  // according to 'intra_partition_tuple_row_less_than_'.
-  int cmp_result = node->tuple_row_less_than_->Compare(materialized_tuple, top_tuple);
+  // according to 'cmp'.
+  int cmp_result = cmp.Compare(materialized_tuple, top_tuple);
   if (cmp_result == 0) {
     // This is a duplicate of the current min, we need to include it as a tie with min.
     Tuple* insert_tuple =
@@ -96,8 +102,7 @@ int TopNNode::Heap::InsertTupleWithTieHandling(
     priority_queue_.Pop();
 
     // Check if 'top_tuple' (the tuple we just popped off) is tied with the new head.
-    if (heap_capacity() > 1 &&
-        node->tuple_row_less_than_->Compare(top_tuple, priority_queue_.Top()) == 0) {
+    if (heap_capacity() > 1 && cmp.Compare(top_tuple, priority_queue_.Top()) == 0) {
       // The new top is still tied with the tuples in 'overflowed_ties_' so we must keep
       // it. The previous top becomes another overflowed tuple.
       overflowed_ties_.push_back(top_tuple);
@@ -119,3 +124,65 @@ int TopNNode::Heap::InsertTupleWithTieHandling(
   }
 }
 
+void TopNNode::InsertBatchPartitioned(RuntimeState* state, RowBatch* batch) {
+  DCHECK(is_partitioned());
+  // Insert all of the rows in the batch into the per-partition heaps. The soft memory
+  // limit will be checked later, in case this batch put us over the limit.
+  FOREACH_ROW(batch, 0, iter) {
+    tmp_tuple_->MaterializeExprs<false, true>(
+        iter.Get(), *output_tuple_desc_, output_tuple_expr_evals_, nullptr);
+    // TODO: IMPALA-10228: the comparator won't get inlined by codegen here.
+    auto it = partition_heaps_.find(tmp_tuple_);
+    Heap* new_heap = nullptr;
+    Heap* heap;
+    if (it == partition_heaps_.end()) {
+      // Allocate the heap here, but insert in into partition_heaps_ later once we've
+      // initialized the tuple that will be the key.
+      new_heap =
+          new Heap(*intra_partition_order_cmp_, per_partition_limit(), include_ties());
+      heap = new_heap;
+      COUNTER_ADD(in_mem_heap_created_counter_, 1);
+    } else {
+      heap = it->second.get();
+    }
+    heap->InsertMaterializedTuple(this, tmp_tuple_);
+    if (new_heap != nullptr) {
+      DCHECK_GT(new_heap->num_tuples(), 0);
+      // Add the new heap with the first tuple as the key.
+      partition_heaps_.emplace(new_heap->top(), unique_ptr<Heap>(new_heap));
+    }
+  }
+}
+
+void TopNNode::Heap::InsertMaterializedTuple(
+    TopNNode* node, Tuple* materialized_tuple) {
+  DCHECK(node->is_partitioned());
+  const TupleDescriptor& tuple_desc = *node->output_tuple_desc_;
+  Tuple* insert_tuple = nullptr;
+  if (priority_queue_.Size() < heap_capacity()) {
+    // Add all tuples until we hit capacity.
+    insert_tuple =
+        reinterpret_cast<Tuple*>(node->tuple_pool_->Allocate(node->tuple_byte_size()));
+    materialized_tuple->DeepCopy(insert_tuple, tuple_desc, node->tuple_pool_.get());
+    priority_queue_.Push(insert_tuple);
+    return;
+  }
+
+  // We're at capacity - compare to the first row in the priority queue to see if
+  // we need to insert this row into the heap.
+  DCHECK(!priority_queue_.Empty());
+  Tuple* top_tuple = priority_queue_.Top();
+  if (!include_ties()) {
+    ++num_tuples_discarded_; // One of the tuples will be discarded.
+    int cmp_result =
+        node->intra_partition_order_cmp_->Compare(materialized_tuple, top_tuple);
+    if (cmp_result >= 0) return;
+    // Pop off the old head, and replace with the new tuple. Reuse the fixed-length
+    // memory of 'top_tuple' to reduce allocations.
+    materialized_tuple->DeepCopy(top_tuple, tuple_desc, node->tuple_pool_.get());
+    priority_queue_.HeapifyFromTop();
+    return;
+  }
+  num_tuples_discarded_ += InsertTupleWithTieHandling(
+      *node->intra_partition_order_cmp_, node, materialized_tuple);
+}
diff --git a/be/src/exec/topn-node.cc b/be/src/exec/topn-node.cc
index 1ea16e8..fc43186 100644
--- a/be/src/exec/topn-node.cc
+++ b/be/src/exec/topn-node.cc
@@ -17,18 +17,22 @@
 
 #include "exec/topn-node.h"
 
+#include <algorithm>
 #include <sstream>
 
 #include "codegen/llvm-codegen.h"
 #include "exec/exec-node-util.h"
-#include "exprs/scalar-expr.h"
 #include "exprs/scalar-expr-evaluator.h"
+#include "exprs/scalar-expr.h"
+#include "exprs/slot-ref.h"
 #include "runtime/descriptors.h"
 #include "runtime/fragment-state.h"
 #include "runtime/mem-pool.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
+#include "runtime/sorter.h"
+#include "runtime/sorter-internal.h" // For TupleSorter
 #include "runtime/tuple-row.h"
 #include "runtime/tuple.h"
 #include "util/debug-util.h"
@@ -42,6 +46,19 @@
 
 using namespace impala;
 
+// Soft limit on the number of partitions in an instance of a partitioned top-n
+// node to avoid potential scalability problems with a large number of heaps in
+// the std::map.
+DEFINE_int32(partitioned_topn_in_mem_partitions_limit, 1000, "(Experimental) Soft limit "
+    "on the number of in-memory partitions in an instance of the partitioned top-n "
+    "operator.");
+
+// Soft limit on the aggregate size of heaps. If heaps exceed this, we will evict some
+// from memory.
+DEFINE_int64(partitioned_topn_soft_limit_bytes, 64L * 1024L * 1024L, "(Experimental) "
+    "Soft limit on the number of in-memory partitions in an instance of the "
+    "partitioned top-n operator.");
+
 Status TopNPlanNode::Init(const TPlanNode& tnode, FragmentState* state) {
   const TSortInfo& tsort_info = tnode.sort_node.sort_info;
   RETURN_IF_ERROR(PlanNode::Init(tnode, state));
@@ -51,8 +68,39 @@ Status TopNPlanNode::Init(const TPlanNode& tnode, FragmentState* state) {
   output_tuple_desc_ = row_descriptor_->tuple_descriptors()[0];
   RETURN_IF_ERROR(ScalarExpr::Create(tsort_info.sort_tuple_slot_exprs,
       *children_[0]->row_descriptor_, state, &output_tuple_exprs_));
-  row_comparator_config_ =
+  ordering_comparator_config_ =
       state->obj_pool()->Add(new TupleRowComparatorConfig(tsort_info, ordering_exprs_));
+  if (is_partitioned()) {
+    DCHECK(tnode.sort_node.__isset.partition_exprs);
+    RETURN_IF_ERROR(ScalarExpr::Create(
+        tnode.sort_node.partition_exprs, *row_descriptor_, state, &partition_exprs_));
+
+    // We need a TSortInfo for internal use in the sorted map. Initialize with
+    // arbitrary parameters.
+    TSortInfo* tpartition_sort_info = state->obj_pool()->Add(new TSortInfo);
+    tpartition_sort_info->sorting_order = TSortingOrder::LEXICAL;
+    tpartition_sort_info->is_asc_order.resize(partition_exprs_.size(), true);
+    tpartition_sort_info->nulls_first.resize(partition_exprs_.size(), false);
+    partition_comparator_config_ = state->obj_pool()->Add(
+        new TupleRowComparatorConfig(*tpartition_sort_info, partition_exprs_));
+
+    DCHECK(tnode.sort_node.__isset.intra_partition_sort_info);
+    const TSortInfo& intra_part_sort_info = tnode.sort_node.intra_partition_sort_info;
+    // Set up the intra-partition comparator.
+    RETURN_IF_ERROR(ScalarExpr::Create(intra_part_sort_info.ordering_exprs,
+        *row_descriptor_, state, &intra_partition_ordering_exprs_));
+    intra_partition_comparator_config_ =
+        state->obj_pool()->Add(new TupleRowComparatorConfig(
+            intra_part_sort_info, intra_partition_ordering_exprs_));
+
+    // Construct SlotRefs that simply copy the output tuple to itself.
+    for (const SlotDescriptor* slot_desc : output_tuple_desc_->slots()) {
+      SlotRef* slot_ref =
+          state->obj_pool()->Add(new SlotRef(slot_desc, slot_desc->type()));
+      noop_tuple_exprs_.push_back(slot_ref);
+      RETURN_IF_ERROR(slot_ref->Init(*row_descriptor_, true, state));
+    }
+  }
   DCHECK_EQ(conjuncts_.size(), 0) << "TopNNode should never have predicates to evaluate.";
   state->CheckAndAddCodegenDisabledMessage(codegen_status_msgs_);
   return Status::OK();
@@ -60,7 +108,10 @@ Status TopNPlanNode::Init(const TPlanNode& tnode, FragmentState* state) {
 
 void TopNPlanNode::Close() {
   ScalarExpr::Close(ordering_exprs_);
+  ScalarExpr::Close(partition_exprs_);
+  ScalarExpr::Close(intra_partition_ordering_exprs_);
   ScalarExpr::Close(output_tuple_exprs_);
+  ScalarExpr::Close(noop_tuple_exprs_);
   PlanNode::Close();
 }
 
@@ -76,11 +127,16 @@ TopNNode::TopNNode(
     offset_(pnode.offset()),
     output_tuple_exprs_(pnode.output_tuple_exprs_),
     output_tuple_desc_(pnode.output_tuple_desc_),
-    tuple_row_less_than_(new TupleRowLexicalComparator(*pnode.row_comparator_config_)),
+    order_cmp_(new TupleRowLexicalComparator(*pnode.ordering_comparator_config_)),
+    partition_cmp_(pnode.partition_comparator_config_ == nullptr ?
+            nullptr :
+            new TupleRowLexicalComparator(*pnode.partition_comparator_config_)),
+    intra_partition_order_cmp_(pnode.intra_partition_comparator_config_ == nullptr ?
+            nullptr :
+            new TupleRowLexicalComparator(*pnode.intra_partition_comparator_config_)),
     tuple_pool_(nullptr),
     codegend_insert_batch_fn_(pnode.codegend_insert_batch_fn_),
-    rows_to_reclaim_(0),
-    num_rows_skipped_(0) {
+    partition_heaps_(ComparatorWrapper<TupleRowComparator>(*partition_cmp_)) {
   runtime_profile()->AddInfoString("SortType", "TopN");
 }
 
@@ -94,6 +150,32 @@ Status TopNNode::Prepare(RuntimeState* state) {
   insert_batch_timer_ = ADD_TIMER(runtime_profile(), "InsertBatchTime");
   tuple_pool_reclaim_counter_ = ADD_COUNTER(runtime_profile(), "TuplePoolReclamations",
       TUnit::UNIT);
+  if (is_partitioned()) {
+    num_partitions_counter_ = ADD_COUNTER(runtime_profile(), "NumPartitions",
+        TUnit::UNIT);
+    in_mem_heap_created_counter_ = ADD_COUNTER(runtime_profile(), "InMemoryHeapsCreated",
+        TUnit::UNIT);
+    in_mem_heap_evicted_counter_ = ADD_COUNTER(runtime_profile(), "InMemoryHeapsEvicted",
+        TUnit::UNIT);
+    in_mem_heap_rows_filtered_counter_ = ADD_COUNTER(runtime_profile(),
+        "InMemoryHeapsRowsFiltered", TUnit::UNIT);
+  }
+
+  // Set up heaps and sorters for the partitioned and non-partitioned cases.
+  const TopNPlanNode& pnode = static_cast<const TopNPlanNode&>(plan_node_);
+  if (is_partitioned()) {
+    DCHECK_GT(per_partition_limit(), 0)
+        << "Planner should not generate partitioned top-n with 0 limit";
+    // Partitioned Top-N needs the external sorter.
+    sorter_.reset(new Sorter(*pnode.ordering_comparator_config_, pnode.noop_tuple_exprs_,
+        &row_descriptor_, mem_tracker(), buffer_pool_client(),
+        resource_profile_.spillable_buffer_size, runtime_profile(), state, label(),
+        true, pnode.codegend_sort_helper_fn_));
+    RETURN_IF_ERROR(sorter_->Prepare(pool_));
+    DCHECK_GE(resource_profile_.min_reservation, sorter_->ComputeMinReservation());
+  } else {
+    heap_.reset(new Heap(*order_cmp_, pnode.heap_capacity(), pnode.include_ties()));
+  }
   return Status::OK();
 }
 
@@ -105,21 +187,38 @@ void TopNPlanNode::Codegen(FragmentState* state) {
   DCHECK(codegen != NULL);
 
   llvm::Function* compare_fn = nullptr;
-  Status codegen_status = row_comparator_config_->Codegen(state, &compare_fn);
+  llvm::Function* intra_partition_compare_fn = nullptr;
+  Status codegen_status = ordering_comparator_config_->Codegen(state, &compare_fn);
+  if (codegen_status.ok() && is_partitioned()) {
+    codegen_status =
+        Sorter::TupleSorter::Codegen(state, compare_fn, &codegend_sort_helper_fn_);
+  }
+  if (codegen_status.ok() && is_partitioned()) {
+    // TODO: IMPALA-10228: replace comparisons in std::map.
+    codegen_status = partition_comparator_config_->Codegen(state);
+  }
+  if (codegen_status.ok() && is_partitioned()) {
+    codegen_status =
+        intra_partition_comparator_config_->Codegen(state, &intra_partition_compare_fn);
+  }
   if (codegen_status.ok()) {
-    llvm::Function* insert_batch_fn =
-        codegen->GetFunction(IRFunction::TOPN_NODE_INSERT_BATCH, true);
+    llvm::Function* insert_batch_fn = codegen->GetFunction(is_partitioned() ?
+            IRFunction::TOPN_NODE_INSERT_BATCH_PARTITIONED :
+            IRFunction::TOPN_NODE_INSERT_BATCH_UNPARTITIONED, true);
     DCHECK(insert_batch_fn != NULL);
 
     // Generate two MaterializeExprs() functions, one with no pool that
-    // does a shallow copy and one with 'tuple_pool_' that does a deep copy of the data.
+    // does a shallow copy (used in partitioned and unpartitioned modes) and
+    // one with 'tuple_pool_' that does a deep copy of the data.
     DCHECK(output_tuple_desc_ != NULL);
     llvm::Function* materialize_exprs_tuple_pool_fn = nullptr;
     llvm::Function* materialize_exprs_no_pool_fn = nullptr;
 
-    codegen_status = Tuple::CodegenMaterializeExprs(codegen, false,
-        *output_tuple_desc_, output_tuple_exprs_,
-        true, &materialize_exprs_tuple_pool_fn);
+    if (!is_partitioned()) {
+      codegen_status = Tuple::CodegenMaterializeExprs(codegen, false,
+          *output_tuple_desc_, output_tuple_exprs_,
+          true, &materialize_exprs_tuple_pool_fn);
+    }
 
     if (codegen_status.ok()) {
       codegen_status = Tuple::CodegenMaterializeExprs(codegen, false,
@@ -129,21 +228,34 @@ void TopNPlanNode::Codegen(FragmentState* state) {
 
     if (codegen_status.ok()) {
       int replaced;
-      replaced = codegen->ReplaceCallSites(insert_batch_fn,
-          materialize_exprs_tuple_pool_fn, Tuple::MATERIALIZE_EXPRS_SYMBOL);
-      DCHECK_REPLACE_COUNT(replaced, 1) << LlvmCodeGen::Print(insert_batch_fn);
+      if (!is_partitioned()) {
+        replaced = codegen->ReplaceCallSites(insert_batch_fn,
+            materialize_exprs_tuple_pool_fn, Tuple::MATERIALIZE_EXPRS_SYMBOL);
+        DCHECK_REPLACE_COUNT(replaced, 1) << LlvmCodeGen::Print(insert_batch_fn);
+      }
 
       replaced = codegen->ReplaceCallSites(insert_batch_fn,
           materialize_exprs_no_pool_fn, Tuple::MATERIALIZE_EXPRS_NULL_POOL_SYMBOL);
       DCHECK_REPLACE_COUNT(replaced, 1) << LlvmCodeGen::Print(insert_batch_fn);
 
-      // The total number of calls to tuple_row_less_than_->Compare() is 3 in
-      // PriorityQueue (called from 2 places), 1 in TopNNode::Heap::InsertTupleRow()
-      // and 3 in TopNNode::Heap::InsertTupleWithTieHandling
-      // Each tuple_row_less_than_->Less(Tuple*, Tuple*) indirectly calls Compare() once.
-      replaced = codegen->ReplaceCallSites(insert_batch_fn,
-          compare_fn, TupleRowComparator::COMPARE_SYMBOL);
-      DCHECK_REPLACE_COUNT(replaced, 10) << LlvmCodeGen::Print(insert_batch_fn);
+      if (is_partitioned()) {
+        // The total number of calls to tuple_row_less_than_->Compare() is 3 in
+        // PriorityQueue (called from 2 places), 1 in
+        // TopNNode::Heap::InsertMaterializedTuple() and 3 in
+        // TopNNode::Heap::InsertTupleWithTieHandling()
+        // Each Less(Tuple*, Tuple*) indirectly calls Compare() once.
+        replaced = codegen->ReplaceCallSites(insert_batch_fn,
+            intra_partition_compare_fn, TupleRowComparator::COMPARE_SYMBOL);
+        DCHECK_REPLACE_COUNT(replaced, 10) << LlvmCodeGen::Print(insert_batch_fn);
+      } else {
+        // The total number of calls to tuple_row_less_than_->Compare() is 3 in
+        // PriorityQueue (called from 2 places), 1 in TopNNode::Heap::InsertTupleRow()
+        // and 3 in TopNNode::Heap::InsertTupleWithTieHandling
+        // Each Less(Tuple*, Tuple*) indirectly calls Compare() once.
+        replaced = codegen->ReplaceCallSites(insert_batch_fn,
+            compare_fn, TupleRowComparator::COMPARE_SYMBOL);
+        DCHECK_REPLACE_COUNT(replaced, 10) << LlvmCodeGen::Print(insert_batch_fn);
+      }
 
       replaced = codegen->ReplaceCallSitesWithValue(insert_batch_fn,
           codegen->GetI64Constant(heap_capacity()), "heap_capacity");
@@ -173,23 +285,33 @@ Status TopNNode::Open(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   ScopedOpenEventAdder ea(this);
   RETURN_IF_ERROR(ExecNode::Open(state));
-  RETURN_IF_ERROR(
-      tuple_row_less_than_->Open(pool_, state, expr_perm_pool(), expr_results_pool()));
-  RETURN_IF_ERROR(ScalarExprEvaluator::Open(output_tuple_expr_evals_, state));
   RETURN_IF_CANCELLED(state);
   RETURN_IF_ERROR(QueryMaintenance(state));
 
-  const TopNPlanNode& pnode = static_cast<const TopNPlanNode&>(plan_node_);
-  heap_.reset(
-      new Heap(*tuple_row_less_than_, pnode.heap_capacity(), pnode.include_ties()));
+  RETURN_IF_ERROR(child(0)->Open(state));
+
+  RETURN_IF_ERROR(
+      order_cmp_->Open(pool_, state, expr_perm_pool(), expr_results_pool()));
+  RETURN_IF_ERROR(ScalarExprEvaluator::Open(output_tuple_expr_evals_, state));
+  if (is_partitioned()) {
+    // Set up state required by partitioned top-N implementation. Claim reservation
+    // after the child has been opened to reduce the peak reservation requirement.
+    if (!buffer_pool_client()->is_registered()) {
+      RETURN_IF_ERROR(ClaimBufferReservation(state));
+    }
+    RETURN_IF_ERROR(
+        partition_cmp_->Open(pool_, state, expr_perm_pool(), expr_results_pool()));
+    RETURN_IF_ERROR(intra_partition_order_cmp_->Open(
+          pool_, state, expr_perm_pool(), expr_results_pool()));
+    RETURN_IF_ERROR(sorter_->Open());
+  }
 
   // Allocate memory for a temporary tuple.
   tmp_tuple_ = reinterpret_cast<Tuple*>(
       tuple_pool_->Allocate(output_tuple_desc_->byte_size()));
 
-  RETURN_IF_ERROR(child(0)->Open(state));
-
   // Limit of 0, no need to fetch anything from children.
+  const TopNPlanNode& pnode = static_cast<const TopNPlanNode&>(plan_node_);
   if (pnode.heap_capacity() != 0) {
     RowBatch batch(child(0)->row_desc(), state->batch_size(), mem_tracker());
     bool eos;
@@ -200,21 +322,28 @@ Status TopNNode::Open(RuntimeState* state) {
         SCOPED_TIMER(insert_batch_timer_);
         TopNPlanNode::InsertBatchFn insert_batch_fn = codegend_insert_batch_fn_.load();
         if (insert_batch_fn != nullptr) {
-          insert_batch_fn(this, &batch);
+          insert_batch_fn(this, state, &batch);
+        } else if (is_partitioned()) {
+          InsertBatchPartitioned(state, &batch);
         } else {
-          InsertBatch(&batch);
+          InsertBatchUnpartitioned(state, &batch);
         }
-        DCHECK(heap_->DCheckConsistency());
-        if (rows_to_reclaim_ > 2 * pnode.heap_capacity()) {
+        DCHECK(is_partitioned() || heap_->DCheckConsistency());
+        if (is_partitioned()) {
+          if (partition_heaps_.size() > FLAGS_partitioned_topn_in_mem_partitions_limit ||
+            tuple_pool_->total_reserved_bytes() >
+              FLAGS_partitioned_topn_soft_limit_bytes) {
+            RETURN_IF_ERROR(EvictPartitions(state, /*evict_final=*/false));
+          }
+        } else if (rows_to_reclaim_ > 2 * unpartitioned_capacity()) {
           RETURN_IF_ERROR(ReclaimTuplePool(state));
-          COUNTER_ADD(tuple_pool_reclaim_counter_, 1);
         }
       }
       RETURN_IF_CANCELLED(state);
       RETURN_IF_ERROR(QueryMaintenance(state));
     } while (!eos);
   }
-  PrepareForOutput();
+  RETURN_IF_ERROR(PrepareForOutput(state));
 
   // Unless we are inside a subplan expecting to call Open()/GetNext() on the child
   // again, the child can be closed at this point.
@@ -226,6 +355,13 @@ Status TopNNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   ScopedGetNextEventAdder ea(this, eos);
   RETURN_IF_ERROR(ExecDebugAction(TExecNodePhase::GETNEXT, state));
+  return is_partitioned() ? GetNextPartitioned(state, row_batch, eos)
+                          : GetNextUnpartitioned(state, row_batch, eos);
+}
+
+Status TopNNode::GetNextUnpartitioned(
+    RuntimeState* state, RowBatch* row_batch, bool* eos) {
+  DCHECK(!is_partitioned());
   RETURN_IF_CANCELLED(state);
   RETURN_IF_ERROR(QueryMaintenance(state));
   while (!row_batch->AtCapacity() && (get_next_iter_ != sorted_top_n_.end())) {
@@ -242,7 +378,6 @@ Status TopNNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
     ++get_next_iter_;
     row_batch->CommitLastRow();
     IncrementNumRowsReturned(1);
-    COUNTER_SET(rows_returned_counter_, rows_returned());
   }
   *eos = get_next_iter_ == sorted_top_n_.end();
 
@@ -251,11 +386,98 @@ Status TopNNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
   // inside a subplan, we might choose to only selectively transfer, e.g., when the
   // block(s) in the pool are all full or when the pool has reached a certain size.
   if (*eos) row_batch->tuple_data_pool()->AcquireData(tuple_pool_.get(), false);
+  COUNTER_SET(rows_returned_counter_, rows_returned());
+  return Status::OK();
+}
+
+Status TopNNode::GetNextPartitioned(
+    RuntimeState* state, RowBatch* batch, bool* eos) {
+  DCHECK(is_partitioned());
+  *eos = false;
+  while (!batch->AtCapacity()) {
+    RETURN_IF_CANCELLED(state);
+    RETURN_IF_ERROR(QueryMaintenance(state));
+    if (sort_out_batch_pos_ >= sort_out_batch_->num_rows()) {
+      // Output rows will reference tuples from sorter output batches - make sure memory
+      // is transferred correctly.
+      sort_out_batch_->TransferResourceOwnership(batch);
+      sort_out_batch_->Reset();
+      sort_out_batch_pos_ = 0;
+      if (batch->AtCapacity()) break;
+      bool sorter_eos = false;
+      RETURN_IF_ERROR(sorter_->GetNext(sort_out_batch_.get(), &sorter_eos));
+      if (sorter_eos && sort_out_batch_->num_rows() == 0) {
+        sort_out_batch_->TransferResourceOwnership(batch);
+        *eos = true;
+        break;
+      }
+    }
+    // Copy rows within the partition limits from 'sort_out_batch_' to 'batch'.
+    // NOTE: this loop could be codegen'd, but is unlikely to be the bottleneck for
+    // most partitioned top-N queries.
+    while (sort_out_batch_pos_ < sort_out_batch_->num_rows()) {
+      TupleRow* curr_row = sort_out_batch_->GetRow(sort_out_batch_pos_);
+      ++sort_out_batch_pos_;
+      // If 'num_rows_returned_from_partition_' > 0, then 'prev_row' is the previous row
+      // returned from the current partition.
+      TupleRow* prev_row = reinterpret_cast<TupleRow*>(&tmp_tuple_);
+      bool add_row = false;
+      if (num_rows_returned_from_partition_ > 0
+          && partition_cmp_->Compare(curr_row, prev_row) == 0) {
+        // Return rows up to the limit plus any ties that match the last returned row.
+        if (num_rows_returned_from_partition_ < per_partition_limit()
+            || (include_ties() &&
+                intra_partition_order_cmp_->Compare(curr_row, prev_row) == 0)) {
+          add_row = true;
+          ++num_rows_returned_from_partition_;
+        }
+      } else {
+        // New partition.
+        DCHECK_GT(per_partition_limit(), 0);
+        COUNTER_ADD(num_partitions_counter_, 1);
+        add_row = true;
+        num_rows_returned_from_partition_ = 1;
+      }
+      if (add_row) {
+        Tuple* out_tuple = curr_row->GetTuple(0);
+        tmp_tuple_ = out_tuple;
+        TupleRow* out_row = batch->GetRow(batch->AddRow());
+        out_row->SetTuple(0, out_tuple);
+        batch->CommitLastRow();
+        IncrementNumRowsReturned(1);
+        if (batch->AtCapacity()) break;
+      }
+    }
+  }
+  DCHECK(*eos || batch->AtCapacity());
+  if (num_rows_returned_from_partition_ == 0) {
+    // tmp_tuple_ references a previous partition, if anything. Make it clear that it's
+    // invalid.
+    tmp_tuple_ = nullptr;
+  } else if (num_rows_returned_from_partition_ > 0) {
+    // 'tmp_tuple_' is part of the current partition. Deep copy so that it doesn't
+    // reference memory that is attached to the output row batch.
+    Tuple* prev_tmp_tuple = tmp_tuple_;
+    unique_ptr<MemPool> temp_pool(new MemPool(mem_tracker()));
+    RETURN_IF_ERROR(InitTmpTuple(state, temp_pool.get()));
+    prev_tmp_tuple->DeepCopy(tmp_tuple_, *output_tuple_desc_, temp_pool.get());
+    tuple_pool_->FreeAll();
+    tuple_pool_ = move(temp_pool);
+  }
+  COUNTER_SET(rows_returned_counter_, rows_returned());
   return Status::OK();
 }
 
 Status TopNNode::Reset(RuntimeState* state, RowBatch* row_batch) {
-  heap_->Reset();
+  if (is_partitioned()) {
+    partition_heaps_.clear();
+    sorter_->Reset();
+    sort_out_batch_.reset();
+    sort_out_batch_pos_ = 0;
+    num_rows_returned_from_partition_ = 0;
+  } else {
+    heap_->Reset();
+  }
   tmp_tuple_ = nullptr;
   num_rows_skipped_ = 0;
   // Transfer ownership of tuple data to output batch.
@@ -268,20 +490,137 @@ Status TopNNode::Reset(RuntimeState* state, RowBatch* row_batch) {
 void TopNNode::Close(RuntimeState* state) {
   if (is_closed()) return;
   if (heap_ != nullptr) heap_->Close();
+  for (auto& entry : partition_heaps_) {
+    entry.second->Close();
+  }
   if (tuple_pool_.get() != nullptr) tuple_pool_->FreeAll();
-  if (tuple_row_less_than_.get() != nullptr) tuple_row_less_than_->Close(state);
+  if (order_cmp_.get() != nullptr) order_cmp_->Close(state);
+  if (partition_cmp_.get() != nullptr) partition_cmp_->Close(state);
+  if (intra_partition_order_cmp_.get() != nullptr) {
+    intra_partition_order_cmp_->Close(state);
+  }
+  if (sorter_ != nullptr) sorter_->Close(state);
+  sort_out_batch_.reset();
   ScalarExprEvaluator::Close(output_tuple_expr_evals_, state);
   ExecNode::Close(state);
 }
 
-void TopNNode::PrepareForOutput() {
-  DCHECK(heap_->DCheckConsistency());
-  heap_->PrepareForOutput(*this, &sorted_top_n_);
-  get_next_iter_ = sorted_top_n_.begin();
+Status TopNNode::EvictPartitions(RuntimeState* state, bool evict_final) {
+  DCHECK(is_partitioned());
+  vector<unique_ptr<Heap>> heaps_to_evict;
+  if (evict_final) {
+    // Move all the partitions to 'sorter_' in preparation for the final sort. Partitions
+    // are evicted in the order of the partition key to reduce the amount of shuffling
+    // that the final sort will do to rearrange partitions.
+    for (auto& entry : partition_heaps_) {
+      heaps_to_evict.push_back(move(entry.second));
+    }
+    partition_heaps_.clear();
+  } else {
+    heaps_to_evict = SelectPartitionsToEvict();
+  }
+  // Only count heap eviction if they are as a result of memory pressure.
+  if (!evict_final) COUNTER_ADD(in_mem_heap_evicted_counter_, heaps_to_evict.size());
+
+  RowBatch batch(row_desc(), state->batch_size(), mem_tracker());
+  for (auto& heap : heaps_to_evict) {
+    DCHECK(heap->DCheckConsistency());
+    // Extract partition entries from the heap in sorted order to reduce amount of sorting
+    // required in final sort. This sorting is not required for correctness since
+    // 'sorter_' will do a full sort later.
+    heap->PrepareForOutput(*this, &sorted_top_n_);
+    for (int64_t i = 0; i < sorted_top_n_.size(); ++i) {
+      TupleRow* row = batch.GetRow(batch.AddRow());
+      row->SetTuple(0, sorted_top_n_[i]);
+      batch.CommitLastRow();
+      if (batch.AtCapacity() || i == sorted_top_n_.size() - 1) {
+        RETURN_IF_ERROR(sorter_->AddBatch(&batch));
+        batch.Reset();
+      }
+    }
+    sorted_top_n_.clear();
+  }
+  heaps_to_evict.clear();
+
+  // ReclaimTuplePool() can now reclaim memory that is not used by in-memory partitions.
+  RETURN_IF_ERROR(ReclaimTuplePool(state));
+  return Status::OK();
+}
+
+vector<unique_ptr<TopNNode::Heap>> TopNNode::SelectPartitionsToEvict() {
+  // Evict a subset of heaps to free enough memory to continue.
+  // The goal of this approach is to try to maximize rows filtered out, while only
+  // adding O(1) amortized cost per input row. Rematerializing all the heaps (required
+  // to free memory) is O(m) work, where m is the total number of tuples in the heaps.
+  // If we clear out O(m) tuples, that means we will have to process at least O(m) input
+  // rows before another eviction, so the amortized overhead of eviction per input row
+  // is O(m) / O(m) = O(1).
+  //
+  // We evict heaps starting with the heaps that were least effective at filtering
+  // input. We evict 25% of heap tuples so that we achieve O(1) amortized time but
+  // retain effectively filtering heaps as much as possible. We break ties, which
+  // are most likely heaps that have not filtered input since the last eviction,
+  // based on whether they are growing and likely to start filtering in the near
+  // future.
+  // TODO: it's possible that we could free up memory without evicting any heaps
+  // just by reclaiming unreferenced variable-length data. We do not do that yet
+  // because we don't know if it will reclaim enough memory. Evicting some heaps
+  // is guaranteed to be effective.
+  vector<PartitionHeapMap::iterator> sorted_heaps;
+  int64_t total_tuples = 0;
+  sorted_heaps.reserve(partition_heaps_.size());
+  for (auto it = partition_heaps_.begin(); it != partition_heaps_.end(); ++it) {
+    total_tuples += it->second->num_tuples();
+    sorted_heaps.push_back(it);
+  }
+  sort(sorted_heaps.begin(), sorted_heaps.end(),
+      [](const PartitionHeapMap::iterator& left,
+          const PartitionHeapMap::iterator& right) {
+        int64_t left_discarded = left->second->num_tuples_discarded();
+        int64_t right_discarded = right->second->num_tuples_discarded();
+        if (left_discarded != right_discarded) {
+          return left_discarded < right_discarded;
+        }
+        return left->second->num_tuples_added_since_eviction() <
+            right->second->num_tuples_added_since_eviction();
+      });
+
+  vector<unique_ptr<Heap>> result;
+  int64_t num_tuples_evicted = 0;
+  for (auto it : sorted_heaps) {
+    if (num_tuples_evicted < total_tuples / 4) {
+      result.push_back(move(it->second));
+      partition_heaps_.erase(it);
+      num_tuples_evicted += result.back()->num_tuples();
+    } else {
+      // Reset counters on surviving heaps so that statistics are accurate about
+      // recent filtering.
+      it->second->ResetStats(*this);
+    }
+  }
+  return result;
+}
+
+Status TopNNode::PrepareForOutput(RuntimeState* state) {
+  if (is_partitioned()) {
+    // Dump all rows into the sorter and sort by partition, so that we can iterate
+    // through the rows and build heaps partition-by-partition.
+    RETURN_IF_ERROR(EvictPartitions(state, /*evict_final=*/true));
+    DCHECK(partition_heaps_.empty());
+    RETURN_IF_ERROR(sorter_->InputDone());
+    sort_out_batch_.reset(
+        new RowBatch(row_desc(), state->batch_size(), mem_tracker()));
+  } else {
+    DCHECK(heap_->DCheckConsistency());
+    heap_->PrepareForOutput(*this, &sorted_top_n_);
+    get_next_iter_ = sorted_top_n_.begin();
+  }
+  return Status::OK();
 }
 
 void TopNNode::Heap::PrepareForOutput(
     const TopNNode& RESTRICT node, vector<Tuple*>* sorted_top_n) RESTRICT {
+  ResetStats(node); // Ensure all counters are updated.
   // Reverse the order of the tuples in the priority queue
   sorted_top_n->resize(num_tuples());
   int64_t index = sorted_top_n->size() - 1;
@@ -297,6 +636,13 @@ void TopNNode::Heap::PrepareForOutput(
   }
 }
 
+void TopNNode::Heap::ResetStats(const TopNNode& RESTRICT node) {
+  RuntimeProfile::Counter* counter = node.in_mem_heap_rows_filtered_counter_;
+  if (counter != nullptr) COUNTER_ADD(counter, num_tuples_discarded_);
+  num_tuples_discarded_ = 0;
+  num_tuples_at_last_eviction_ = num_tuples();
+}
+
 bool TopNNode::Heap::DCheckConsistency() {
   DCHECK_LE(num_tuples(), capacity_ + overflowed_ties_.size())
       << num_tuples() << " > " << capacity_ << " + " << overflowed_ties_.size();
@@ -309,20 +655,42 @@ bool TopNNode::Heap::DCheckConsistency() {
 }
 
 Status TopNNode::ReclaimTuplePool(RuntimeState* state) {
+  COUNTER_ADD(tuple_pool_reclaim_counter_, 1);
   unique_ptr<MemPool> temp_pool(new MemPool(mem_tracker()));
-  RETURN_IF_ERROR(heap_->RematerializeTuples(this, state, temp_pool.get()));
-  DCHECK(heap_->DCheckConsistency());
 
+  if (is_partitioned()) {
+    vector<unique_ptr<Heap>> rematerialized_heaps;
+    for (auto& entry : partition_heaps_) {
+      RETURN_IF_ERROR(entry.second->RematerializeTuples(this, state, temp_pool.get()));
+      DCHECK(entry.second->DCheckConsistency());
+      // The key references memory in 'tuple_pool_'. Replace it with a rematerialized
+      // tuple.
+      rematerialized_heaps.push_back(move(entry.second));
+    }
+    partition_heaps_.clear();
+    for (auto& heap_ptr : rematerialized_heaps) {
+      const Tuple* key_tuple = heap_ptr->top();
+      partition_heaps_.emplace(key_tuple, move(heap_ptr));
+    }
+  } else {
+    RETURN_IF_ERROR(heap_->RematerializeTuples(this, state, temp_pool.get()));
+    DCHECK(heap_->DCheckConsistency());
+  }
   rows_to_reclaim_ = 0;
-  tmp_tuple_ = reinterpret_cast<Tuple*>(temp_pool->TryAllocate(
+  RETURN_IF_ERROR(InitTmpTuple(state, temp_pool.get()));
+  tuple_pool_->FreeAll();
+  tuple_pool_ = move(temp_pool);
+  return Status::OK();
+}
+
+Status TopNNode::InitTmpTuple(RuntimeState* state, MemPool* pool) {
+  tmp_tuple_ = reinterpret_cast<Tuple*>(pool->TryAllocate(
       output_tuple_desc_->byte_size()));
   if (UNLIKELY(tmp_tuple_ == nullptr)) {
-    return temp_pool->mem_tracker()->MemLimitExceeded(state,
+    return pool->mem_tracker()->MemLimitExceeded(state,
         "Failed to allocate memory in TopNNode::ReclaimTuplePool.",
         output_tuple_desc_->byte_size());
   }
-  tuple_pool_->FreeAll();
-  tuple_pool_ = move(temp_pool);
   return Status::OK();
 }
 
diff --git a/be/src/exec/topn-node.h b/be/src/exec/topn-node.h
index 9f033c2..c07eb4c 100644
--- a/be/src/exec/topn-node.h
+++ b/be/src/exec/topn-node.h
@@ -24,6 +24,7 @@
 #include "codegen/impala-ir.h"
 #include "exec/exec-node.h"
 #include "runtime/descriptors.h"  // for TupleId
+#include "runtime/sorter.h"
 #include "util/tuple-row-compare.h"
 #include "util/priority-queue.h"
 
@@ -31,6 +32,7 @@ namespace impala {
 
 class MemPool;
 class RuntimeState;
+class Sorter;
 class TopNNode;
 class Tuple;
 
@@ -46,9 +48,28 @@ class TopNPlanNode : public PlanNode {
   }
   ~TopNPlanNode(){}
 
+  /// Return true if this is a partitioned Top-N.
+  bool is_partitioned() const {
+    return tnode_->sort_node.type == TSortType::PARTITIONED_TOPN;
+  }
+
+  /// Returns the per-partition limit.
+  int64_t per_partition_limit() const {
+    DCHECK(is_partitioned());
+    return tnode_->sort_node.per_partition_limit;
+  }
+
   /// Returns the per-heap capacity used for the Heap objects in this node.
   int64_t heap_capacity() const {
-    int64_t limit = include_ties() ? tnode_->sort_node.limit_with_ties : tnode_->limit;
+    int64_t limit;
+    if (is_partitioned()) {
+      limit = per_partition_limit();
+    } else {
+      // Without tie handling, the node-level limit and the heap limit are one and the
+      // same, but with ties they are different because the heap capacity is not
+      // a strict limit on rows.
+      limit = include_ties() ? tnode_->sort_node.limit_with_ties : tnode_->limit;
+    }
     return limit + offset();
   }
 
@@ -59,18 +80,41 @@ class TopNPlanNode : public PlanNode {
   /// Ordering expressions used for tuple comparison.
   std::vector<ScalarExpr*> ordering_exprs_;
 
+  /// Partitioning expressions used for tuple comparison. Non-empty if this is a
+  /// partitioned top N.
+  std::vector<ScalarExpr*> partition_exprs_;
+
+  /// Ordering expressions used for tuple comparison within partition.
+  std::vector<ScalarExpr*> intra_partition_ordering_exprs_;
+
   /// Cached descriptor for the materialized tuple.
   TupleDescriptor* output_tuple_desc_ = nullptr;
 
   /// Materialization exprs for the output tuple and their evaluators.
   std::vector<ScalarExpr*> output_tuple_exprs_;
 
+  /// Materialization exprs that materialize the output tuple into itself, i.e. are
+  /// no-ops. Non-empty if this is a partitioned top N.
+  std::vector<ScalarExpr*> noop_tuple_exprs_;
+
   /// Config used to create a TupleRowComparator instance for 'ordering_exprs_'.
-  TupleRowComparatorConfig* row_comparator_config_ = nullptr;
+  TupleRowComparatorConfig* ordering_comparator_config_ = nullptr;
 
-  /// Codegened version of TopNNode::InsertBatch().
-  typedef void (*InsertBatchFn)(TopNNode*, RowBatch*);
+  /// Config used to create a TupleRowComparator instance for 'partition_exprs_'.
+  /// Non-NULL iff this is a partitioned top N.
+  TupleRowComparatorConfig* partition_comparator_config_ = nullptr;
+
+  /// Config used to create a TupleRowComparator instance for
+  /// 'intra_partition_ordering_exprs_'.
+  TupleRowComparatorConfig* intra_partition_comparator_config_ = nullptr;
+
+  /// Codegened version of TopNNode::InsertBatchUnpartitioned() or
+  /// InsertBatchPartitioned().
+  typedef void (*InsertBatchFn)(TopNNode*, RuntimeState*, RowBatch*);
   CodegenFnPtr<InsertBatchFn> codegend_insert_batch_fn_;
+
+  /// Codegened version of Sort::TupleSorter::SortHelper().
+  CodegenFnPtr<Sorter::SortHelperFn> codegend_sort_helper_fn_;
 };
 
 /// Node for in-memory TopN operator that sorts input tuples and applies a limit such
@@ -83,8 +127,14 @@ class TopNPlanNode : public PlanNode {
 /// rows that are not in the top N as soon as possible, minimizing the memory requirements
 /// and processing time of the operator.
 ///
-/// TODO: currently we only support a single top-n heap per operator. IMPALA-9979 will
-/// add a partitioned mode.
+/// TopNNode supports two modes: unpartitioned and partitioned. In unpartitioned mode,
+/// there is a global limit to the rows returned. Whereas in partitioned mode, tuples
+/// are divided into different partitions based on 'partition_cmp_' and the Top N from
+/// each partition are returned.
+///
+/// In both unpartitioned and partitioned mode, rows are returned fully sorted according
+/// to the sort order (i.e. in the partitioned case, sorted by partition then
+/// intra-partition order).
 ///
 /// Unpartitioned TopN Implementation Details
 /// =========================================
@@ -93,6 +143,21 @@ class TopNPlanNode : public PlanNode {
 /// rows can be directly outputted from the priority queue by calling
 /// Heap::PrepareForOutput().
 ///
+/// Partitioned Top-N Implementation Details
+/// ========================================
+/// Partitioned mode needs to support spilling to disk because the number of partitions
+/// is not known ahead of time, so memory requirements are not known ahead of time. In
+/// partitioned mode, a separate in-memory heap per partition is maintained, until a
+/// soft memory limit is reached, in which case rows are moved to an external Sorter
+/// to stay under the memory limit. Even if the operator is forced to move rows to
+/// the external sorter, the in-memory heaps may still be effective at reducing the
+/// input considerably.
+///
+/// After all the input is consumed, all rows from the in-memory heaps are moved to the
+/// sorter, fully sorted by partition and intra-partition order, after which the rows
+/// can be fetched in order from the sorter - see PrepareForOutput() and
+/// GetNextPartitioned().
+///
 /// Memory Management
 /// =================
 /// In-memory heaps are backed by 'tuple_pool_' - all tuples in the heaps must reference
@@ -104,6 +169,11 @@ class TopNPlanNode : public PlanNode {
 /// In unpartitioned mode, reclamation is triggered by 'rows_to_reclaim_' hitting a
 /// threshold, which indicates that enough unused memory may have accumulated to
 /// be worth reclaiming.
+///
+/// In partitioned mode, reclamation is triggered by a memory threshold, after which
+/// some in-memory heaps are evicted and the remaining heaps reclaimed. The reclamation
+/// thus serves two purposes: to support spilling-to-disk where we can't fit all
+/// heaps in memory, and to reclaim unused memory.
 class TopNNode : public ExecNode {
  public:
   TopNNode(ObjectPool* pool, const TopNPlanNode& pnode, const DescriptorTbl& descs);
@@ -122,23 +192,68 @@ class TopNNode : public ExecNode {
 
   friend class TupleLessThan;
 
+  /// Return true if this is a partitioned Top-N.
+  bool is_partitioned() const {
+    const TopNPlanNode& pnode = static_cast<const TopNPlanNode&>(plan_node_);
+    return pnode.is_partitioned();
+  }
+
+  int64_t unpartitioned_capacity() const {
+    DCHECK(!is_partitioned());
+    return limit_ + offset_;
+  }
+
+  /// Returns the per-partition limit.
+  int64_t per_partition_limit() const {
+    DCHECK(is_partitioned());
+    const TopNPlanNode& pnode = static_cast<const TopNPlanNode&>(plan_node_);
+    return pnode.per_partition_limit();
+  }
+
   bool include_ties() const {
     const TopNPlanNode& pnode = static_cast<const TopNPlanNode&>(plan_node_);
     return pnode.include_ties();
   }
 
-  /// Inserts all the input rows in 'batch' into 'heap_'.
-  void InsertBatch(RowBatch* batch);
+  /// Inserts all the input rows in 'batch' into 'heap_'. Used for unpartitioned
+  /// Top-N only.
+  void InsertBatchUnpartitioned(RuntimeState* state, RowBatch* batch);
+
+  /// Inserts all the input rows in 'batch' into 'partition_heaps_'. Used for partitioned
+  /// Top-N only.
+  void InsertBatchPartitioned(RuntimeState* state, RowBatch* batch);
+
+  /// Evict some of the partitions from memory, putting the tuples into 'sorter_'. If
+  /// 'evict_final' is true, all partitions will be evicted as part of PrepareOutput().
+  /// Used for partitioned Top-N only.
+  Status EvictPartitions(RuntimeState* state, bool evict_final);
+
+  /// Select a subset of partitions to evict as a result of memory pressure. Removes
+  /// the partitions from 'partition_heaps_' and returns them.
+  /// Used for partitioned Top-N only.
+  std::vector<std::unique_ptr<Heap>> SelectPartitionsToEvict();
+
+  /// Implementation of GetNext() for when is_partitioned() is false.
+  Status GetNextUnpartitioned(RuntimeState* state, RowBatch* row_batch, bool* eos);
+
+  /// Implementation of GetNext() for when is_partitioned() is true.
+  Status GetNextPartitioned(RuntimeState* state, RowBatch* row_batch, bool* eos);
 
   /// Prepare to start outputting rows. Called after consuming all rows from the child.
-  /// Collects all output rows in 'sorted_top_n_' and initializes 'get_next_iter_' to
-  /// point to the first row.
-  void PrepareForOutput();
+  /// In partitioned mode, this adds all rows to 'sorter_' and sorts them so that
+  /// GetNextPartitioned() can fetch the rows in order.
+  ///
+  /// In unpartitioned mode, this collects all output rows in 'sorted_top_n_' and
+  /// initializes 'get_next_iter_' to point to the first row.
+  Status PrepareForOutput(RuntimeState* state);
 
   /// Re-materialize all tuples that reference 'tuple_pool_' and release 'tuple_pool_',
   /// replacing it with a new pool.
   Status ReclaimTuplePool(RuntimeState* state);
 
+  /// Initialize 'tmp_tuple_' with memory from 'pool'.
+  Status InitTmpTuple(RuntimeState* state, MemPool* pool);
+
   IR_NO_INLINE int tuple_byte_size() const noexcept {
     return output_tuple_desc_->byte_size();
   }
@@ -154,7 +269,15 @@ class TopNNode : public ExecNode {
   TupleDescriptor* const output_tuple_desc_;
 
   /// Comparator for ordering tuples globally.
-  std::unique_ptr<TupleRowComparator> tuple_row_less_than_;
+  std::unique_ptr<TupleRowComparator> order_cmp_;
+
+  /// Comparator for partitioning tuples between priority queue. Non-NULL iff this is a
+  /// partitioned top-N operator.
+  std::unique_ptr<TupleRowComparator> partition_cmp_;
+
+  /// Comparator for partitioning tuples within a partition priority queue. Non-NULL iff
+  /// this is a partitioned top-N operator.
+  std::unique_ptr<TupleRowComparator> intra_partition_order_cmp_;
 
   /// Temporary staging vector for sorted tuples extracted from a Heap via
   /// Heap::PrepareForOutput().
@@ -163,35 +286,75 @@ class TopNNode : public ExecNode {
   /// Stores everything referenced in priority_queue_.
   std::unique_ptr<MemPool> tuple_pool_;
 
-  /// Iterator over elements in sorted_top_n_.
+  /// Iterator over elements in sorted_top_n_. Only used in unpartitioned Top-N.
   std::vector<Tuple*>::iterator get_next_iter_;
 
   /// Reference to the codegened function pointer owned by the TopNPlanNode object that
   /// was used to create this instance.
   const CodegenFnPtr<TopNPlanNode::InsertBatchFn>& codegend_insert_batch_fn_;
 
-  /// Timer for time spent in InsertBatch() function (or codegen'd version).
+  /// Timer for time spent in InsertBatch*() function (or codegen'd version).
   RuntimeProfile::Counter* insert_batch_timer_;
 
   /// Number of rows to be reclaimed since tuple_pool_ was last created/reclaimed.
-  int64_t rows_to_reclaim_;
+  /// Only used for unpartitioned Top-N.
+  int64_t rows_to_reclaim_ = 0;
 
   /// Number of times tuple pool memory was reclaimed
   RuntimeProfile::Counter* tuple_pool_reclaim_counter_= nullptr;
 
+  /// Total number of partitions. Only initialized for partitioned Top-N.
+  RuntimeProfile::Counter* num_partitions_counter_ = nullptr;
+
+  /// Number of times an in-memory heap was created.
+  /// Only initialized for partitioned Top-N.
+  RuntimeProfile::Counter* in_mem_heap_created_counter_ = nullptr;
+
+  /// Number of times an in-memory heap was evicted because of memory pressure.
+  /// Only initialized for partitioned Top-N.
+  RuntimeProfile::Counter* in_mem_heap_evicted_counter_ = nullptr;
+
+  /// Number of rows that the in-memory heaps filtered out.
+  /// Only initialized for partitioned Top-N.
+  RuntimeProfile::Counter* in_mem_heap_rows_filtered_counter_ = nullptr;
+
   /////////////////////////////////////////
   /// BEGIN: Members that must be Reset()
 
   /// Tuple allocated once from tuple_pool_ and reused in InsertTupleRow to
   /// materialize input tuples if necessary. After materialization, tmp_tuple_ may be
   /// copied into the tuple pool and inserted into the priority queue.
+  /// Also used in GetNextPartitioned() to store the last output tuple.
   Tuple* tmp_tuple_ = nullptr;
 
-  // Single heap used as the main heap in unpartitioned Top-N.
+  // Single heap used as the main heap in unpartitioned Top-N. Not used for partitioned
+  // Top-N.
   std::unique_ptr<Heap> heap_;
 
+  /// Per-partition heaps used for partitioned Top-N. The map key is a tuple within the
+  /// heap, and 'intra_partition_order_cmp_' is used for comparison.
+  using PartitionHeapMap = std::map<const Tuple*, std::unique_ptr<Heap>,
+      ComparatorWrapper<TupleRowComparator>>;
+  PartitionHeapMap partition_heaps_;
+
   /// Number of rows skipped. Used for adhering to offset_ in unpartitioned Top-N.
-  int64_t num_rows_skipped_;
+  int64_t num_rows_skipped_ = 0;
+
+  /// Sorter used for external sorting. Only used in partitioned Top-N, where tuples are
+  /// sorted by the partition exprs, then the intra-partition ordering exprs.
+  /// Initialized in Prepare().
+  std::unique_ptr<Sorter> sorter_;
+
+  /// Temporary batch used for processing output from sorter in GetNextPartitioned().
+  /// Used only for partitioned Top-N.
+  std::unique_ptr<RowBatch> sort_out_batch_;
+
+  /// Position in 'sort_out_batch_'. Used only for partitioned Top-N.
+  int64_t sort_out_batch_pos_ = 0;
+
+  /// Number of rows returned from the current partition in GetNextPartitioned().
+  /// Used only for partitioned Top-N.
+  int64_t num_rows_returned_from_partition_ = 0;
 
   /// END: Members that must be Reset()
   /////////////////////////////////////////
@@ -210,8 +373,16 @@ class TopNNode::Heap {
   /// copy of 'tuple_row', which it stores in 'tuple_pool'. Always inlined in IR into
   /// TopNNode::InsertBatch() because codegen relies on this for substituting exprs
   /// in the body of TopNNode.
-  /// Returns the number of materialized tuples discarded that may need to be reclaimed.
-  int IR_ALWAYS_INLINE InsertTupleRow(TopNNode* node, TupleRow* input_row);
+  /// Returns the number of rows to be reclaimed.
+  int IR_ALWAYS_INLINE InsertTupleRow(
+      TopNNode* node, TupleRow* input_row) WARN_UNUSED_RESULT;
+
+  /// Insert a tuple row into the priority queue, similar to InsertTupleRow(), except
+  /// 'materialized_row' is already materialized into the output row format, i.e.
+  /// output_tuple_desc_. Always inlined in IR into TopNNode::InsertBatchPartitioned()
+  /// because codegen relies on this for substituting exprs in the body of TopNNode.
+  void IR_ALWAYS_INLINE InsertMaterializedTuple(
+      TopNNode* node, Tuple* materialized_tuple);
 
   /// Copy the elements in the priority queue into a new tuple pool, and release
   /// the previous pool.
@@ -222,6 +393,10 @@ class TopNNode::Heap {
   void PrepareForOutput(
       const TopNNode& RESTRICT node, std::vector<Tuple*>* sorted_top_n) RESTRICT;
 
+  /// Reset stats that are collected about the heap. Called during eviction process in
+  /// partitioned top-N.
+  void ResetStats(const TopNNode& RESTRICT node);
+
   /// Can be called to invoke DCHECKs if the heap is in an inconsistent state.
   /// Returns a bool so it can be wrapped in a DCHECK() macro.
   bool DCheckConsistency();
@@ -229,10 +404,22 @@ class TopNNode::Heap {
   /// Returns number of tuples currently in heap.
   int64_t num_tuples() const { return priority_queue_.Size() + overflowed_ties_.size(); }
 
+  int64_t num_tuples_discarded() const { return num_tuples_discarded_; }
+  int64_t num_tuples_added_since_eviction() const {
+    return num_tuples() - num_tuples_at_last_eviction_;
+  }
+
   IR_NO_INLINE int64_t heap_capacity() const noexcept { return capacity_; }
 
   IR_NO_INLINE bool include_ties() const noexcept { return include_ties_; }
 
+  /// Returns the first element in the priority queue. Should only be called if
+  /// num_tuples() > 0.
+  const Tuple* top() {
+    DCHECK(!priority_queue_.Empty());
+    return priority_queue_.Top();
+  }
+
 private:
   /// Helper for RematerializeTuples() that materializes the tuples in a container in the
   /// range (begin_it, end_it].
@@ -248,8 +435,7 @@ private:
   /// Always inlined in IR because codegen relies on this for substituting exprs in the
   /// body of the function.
   int IR_ALWAYS_INLINE InsertTupleWithTieHandling(
-      TopNNode* node, Tuple* materialized_tuple);
-
+      const TupleRowComparator& cmp, TopNNode* node, Tuple* materialized_tuple);
 
   /// Limit on capacity of 'priority_queue_'. If inserting a tuple into the queue
   /// would exceed this, a tuple is popped off the queue.
@@ -259,6 +445,14 @@ private:
   /// tied to be the head of the heap.
   const bool include_ties_;
 
+  /// Number of tuples discarded as a result of this heap hitting its capacity and
+  /// filtering out tuples. Only updated for the partitioned Top-N.
+  int64_t num_tuples_discarded_ = 0;
+
+  /// Number of tuples in the heap at the time of last eviction. Only updated for the
+  /// partitioned Top-N.
+  int64_t num_tuples_at_last_eviction_ = 0;
+
   /////////////////////////////////////////
   /// BEGIN: Members that must be Reset()
 
diff --git a/be/src/exprs/slot-ref.h b/be/src/exprs/slot-ref.h
index 0449bfb..8f817a4 100644
--- a/be/src/exprs/slot-ref.h
+++ b/be/src/exprs/slot-ref.h
@@ -41,14 +41,14 @@ class SlotRef : public ScalarExpr {
   SlotRef(const TExprNode& node);
   SlotRef(const SlotDescriptor* desc);
 
-  /// TODO: this is a hack to allow aggregation nodes to work around NULL slot
-  /// descriptors. Ideally the FE would dictate the type of the intermediate SlotRefs.
+  /// Instantiate a SlotRef for internal use in the backend (in cases where the frontend
+  /// does not generate the appropriate exprs).
   SlotRef(const SlotDescriptor* desc, const ColumnType& type);
 
   /// Used for testing.  GetValue will return tuple + offset interpreted as 'type'
   SlotRef(const ColumnType& type, int offset, const bool nullable = false);
 
-  /// Exposed as public so AGG node can initialize its build expressions.
+  /// Initialize a SlotRef that was directly constructed by backend.
   virtual Status Init(const RowDescriptor& row_desc, bool is_entry_point,
       FragmentState* state) override WARN_UNUSED_RESULT;
   virtual std::string DebugString() const override;
diff --git a/be/src/service/query-options-test.cc b/be/src/service/query-options-test.cc
index 979f4d5..a6d467d 100644
--- a/be/src/service/query-options-test.cc
+++ b/be/src/service/query-options-test.cc
@@ -280,6 +280,7 @@ TEST(QueryOptions, SetBigIntOptions) {
       {MAKE_OPTIONDEF(cpu_limit_s), {0, I64_MAX}},
       {MAKE_OPTIONDEF(num_rows_produced_limit), {0, I64_MAX}},
       {MAKE_OPTIONDEF(join_rows_produced_limit), {0, I64_MAX}},
+      {MAKE_OPTIONDEF(analytic_rank_pushdown_threshold), {-1, I64_MAX}},
   };
   for (const auto& test_case : case_set) {
     const OptionDef<int64_t>& option_def = test_case.first;
diff --git a/be/src/service/query-options.cc b/be/src/service/query-options.cc
index c7b3a3c..172a6e8 100644
--- a/be/src/service/query-options.cc
+++ b/be/src/service/query-options.cc
@@ -1021,6 +1021,20 @@ Status impala::SetQueryOption(const string& key, const string& value,
         query_options->__set_utf8_mode(IsTrue(value));
         break;
       }
+      case TImpalaQueryOptions::ANALYTIC_RANK_PUSHDOWN_THRESHOLD: {
+        StringParser::ParseResult status;
+        int64_t val =
+            StringParser::StringToInt<int64_t>(value.c_str(), value.size(), &status);
+        if (status != StringParser::PARSE_SUCCESS) {
+          return Status(Substitute("Invalid threshold: '$0'.", value));
+        }
+        if (val < -1) {
+          return Status(Substitute("Invalid threshold: '$0'. Only non-negative values "
+                "and -1 are allowed.", val));
+        }
+        query_options->__set_analytic_rank_pushdown_threshold(val);
+        break;
+      }
       default:
         if (IsRemovedQueryOption(key)) {
           LOG(WARNING) << "Ignoring attempt to set removed query option '" << key << "'";
diff --git a/be/src/service/query-options.h b/be/src/service/query-options.h
index bd787aa..b504177 100644
--- a/be/src/service/query-options.h
+++ b/be/src/service/query-options.h
@@ -47,7 +47,7 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
 // time we add or remove a query option to/from the enum TImpalaQueryOptions.
 #define QUERY_OPTS_TABLE\
   DCHECK_EQ(_TImpalaQueryOptions_VALUES_TO_NAMES.size(),\
-      TImpalaQueryOptions::UTF8_MODE + 1);\
+      TImpalaQueryOptions::ANALYTIC_RANK_PUSHDOWN_THRESHOLD + 1);\
   REMOVED_QUERY_OPT_FN(abort_on_default_limit_exceeded, ABORT_ON_DEFAULT_LIMIT_EXCEEDED)\
   QUERY_OPT_FN(abort_on_error, ABORT_ON_ERROR, TQueryOptionLevel::REGULAR)\
   REMOVED_QUERY_OPT_FN(allow_unsupported_formats, ALLOW_UNSUPPORTED_FORMATS)\
@@ -232,6 +232,8 @@ typedef std::unordered_map<string, beeswax::TQueryOptionLevel::type>
   QUERY_OPT_FN(join_rows_produced_limit, JOIN_ROWS_PRODUCED_LIMIT,\
       TQueryOptionLevel::ADVANCED)\
   QUERY_OPT_FN(utf8_mode, UTF8_MODE, TQueryOptionLevel::DEVELOPMENT)\
+  QUERY_OPT_FN(analytic_rank_pushdown_threshold,\
+      ANALYTIC_RANK_PUSHDOWN_THRESHOLD, TQueryOptionLevel::ADVANCED)\
   ;
 
 /// Enforce practical limits on some query options to avoid undesired query state.
diff --git a/be/src/util/priority-queue.h b/be/src/util/priority-queue.h
index 10b17a4..3ebc098 100644
--- a/be/src/util/priority-queue.h
+++ b/be/src/util/priority-queue.h
@@ -82,14 +82,20 @@ class PriorityQueue {
   void Clear() { elements_.clear(); }
 
   // The top element
-  T& IR_ALWAYS_INLINE Top() { return elements_[0]; }
+  T& IR_ALWAYS_INLINE Top() {
+    DCHECK(!elements_.empty());
+    return elements_[0];
+  }
 
   // The ith element
-  T& operator[](int i) { return elements_[i]; }
+  T& operator[](int i) {
+    DCHECK_LT(i, elements_.size());
+    return elements_[i];
+  }
 
   void Reserve(int64_t capacity) { elements_.reserve(capacity); }
 
-  // The size f the heap
+  // The size of the heap
   int64_t IR_ALWAYS_INLINE Size() const { return elements_.size(); }
 
   // Check if the heap is empty
diff --git a/be/src/util/tuple-row-compare.h b/be/src/util/tuple-row-compare.h
index bf982c4..68e94f5 100644
--- a/be/src/util/tuple-row-compare.h
+++ b/be/src/util/tuple-row-compare.h
@@ -35,6 +35,30 @@ class FragmentState;
 class RuntimeState;
 class ScalarExprEvaluator;
 
+/// A wrapper around types Comparator with a Less() method. This wrapper allows the use of
+/// type Comparator with STL containers which expect a type like std::less<T>, which uses
+/// operator() instead of Less() and is cheap to copy.
+///
+/// The C++ standard requires that std::priority_queue operations behave as wrappers to
+/// {push,pop,make,sort}_heap, which take their comparator object by value. Therefore, it
+/// is inefficient to use comparator objects that have expensive construction,
+/// destruction, and copying with std::priority_queue.
+///
+/// ComparatorWrapper takes a reference to an object of type Comparator, rather than
+/// copying that object. ComparatorWrapper<Comparator>(comp) is not safe to use beyond the
+/// lifetime of comp.
+template <typename Comparator>
+class ComparatorWrapper {
+  const Comparator& comp_;
+ public:
+  ComparatorWrapper(const Comparator& comp) : comp_(comp) {}
+
+  template <typename T>
+  bool operator()(const T& lhs, const T& rhs) const {
+    return comp_.Less(lhs, rhs);
+  }
+};
+
 /// TupleRowComparatorConfig contains the static state initialized from its corresponding
 /// thrift structure. It serves as an input for creating instances of the
 /// TupleRowComparator class.
diff --git a/common/thrift/ImpalaInternalService.thrift b/common/thrift/ImpalaInternalService.thrift
index 781e307..0a6a7e0 100644
--- a/common/thrift/ImpalaInternalService.thrift
+++ b/common/thrift/ImpalaInternalService.thrift
@@ -480,6 +480,9 @@ struct TQueryOptions {
 
   // See comment in ImpalaService.thrift
   121: optional bool utf8_mode = false;
+
+  // See comment in ImpalaService.thrift
+  122: optional i64 analytic_rank_pushdown_threshold = 1000;
 }
 
 // Impala currently has two types of sessions: Beeswax and HiveServer2
diff --git a/common/thrift/ImpalaService.thrift b/common/thrift/ImpalaService.thrift
index 521a604..61f2fb9 100644
--- a/common/thrift/ImpalaService.thrift
+++ b/common/thrift/ImpalaService.thrift
@@ -621,6 +621,15 @@ enum TImpalaQueryOptions {
   // If true, strings are processed in a UTF-8 aware way, e.g. counting lengths by UTF-8
   // characters instead of bytes.
   UTF8_MODE = 120
+
+  // If > 0, the rank()/row_number() pushdown into pre-analytic sorts is enabled
+  // if the limit would be less than or equal to the threshold.
+  // If 0 or -1, disables the optimization (i.e. falls back to pre-Impala-4.0
+  // behaviour).
+  // Default is 1000. Setting it higher increases the max size of the in-memory heaps
+  // used in the top-n operation. The larger the heaps, the less beneficial the
+  // optimization is compared to a full sort and the more potential for perf regressions.
+  ANALYTIC_RANK_PUSHDOWN_THRESHOLD = 121
 }
 
 // The summary of a DML statement.
diff --git a/common/thrift/PlanNodes.thrift b/common/thrift/PlanNodes.thrift
index e6a9bb5..d93224d 100644
--- a/common/thrift/PlanNodes.thrift
+++ b/common/thrift/PlanNodes.thrift
@@ -473,13 +473,16 @@ enum TSortType {
 
   // Divide the input into batches, each of which is sorted individually.
   PARTIAL = 2
+
+  // Return the first N sorted elements from each partition.
+  PARTITIONED_TOPN = 3
 }
 
 struct TSortNode {
   1: required TSortInfo sort_info
   2: required TSortType type
   // This is the number of rows to skip before returning results.
-  // Not used with TSortType::PARTIAL.
+  // Not used with TSortType::PARTIAL or TSortType::PARTITIONED_TOPN.
   3: optional i64 offset
 
   // Estimated bytes of input that will go into this sort node across all backends.
@@ -491,6 +494,16 @@ struct TSortNode {
 
   // If include_ties is true, the limit including ties.
   6: optional i64 limit_with_ties
+
+  // Max number of rows to return per partition.
+  // Used only with TSortType::PARTITIONED_TOPN
+  7: optional i64 per_partition_limit
+  // Used only with TSortType::PARTITIONED_TOPN - expressions over
+  // the sort tuple to use as the partition key.
+  8: optional list<Exprs.TExpr> partition_exprs
+  // Used only with TSortType::PARTITIONED_TOPN - sort info for ordering
+  // within a partition.
+  9: optional TSortInfo intra_partition_sort_info
 }
 
 enum TAnalyticWindowType {
diff --git a/fe/src/main/java/org/apache/impala/analysis/Expr.java b/fe/src/main/java/org/apache/impala/analysis/Expr.java
index de5c411..d3ffab4 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Expr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Expr.java
@@ -1219,6 +1219,17 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
   }
 
   /**
+   * Create a clone of the expression including analysis state with a different
+   * selectivity.
+   */
+  public Expr cloneAndOverrideSelectivity(double selectivity) {
+    Preconditions.checkArgument(selectivity >= 0.0 && selectivity <= 1.0, selectivity);
+    Expr e = clone();
+    e.selectivity_ = selectivity;
+    return e;
+  }
+
+  /**
    * Removes duplicate exprs (according to equals()).
    */
   public static <C extends Expr> void removeDuplicates(List<C> l) {
@@ -1417,6 +1428,19 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
   }
 
   /**
+   * Returns true if this expression tree references a slot in the tuple identified
+   * by tid.
+   */
+  public boolean referencesTuple(TupleId tid) {
+    // This is the default implementation. Expr subclasses that reference slots in
+    // tuples, i.e. SlotRef, must override this.
+    for (Expr child: children_) {
+      if (child.referencesTuple(tid)) return true;
+    }
+    return false;
+  }
+
+  /**
    * Returns true if this expression should be treated as constant. I.e. if the frontend
    * and backend should assume that two evaluations of the expression within a query will
    * return the same value. Examples of constant expressions include:
diff --git a/fe/src/main/java/org/apache/impala/analysis/SlotRef.java b/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
index 22a381b..8c2beed 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
@@ -31,8 +31,8 @@ import org.apache.impala.thrift.TExprNodeType;
 import org.apache.impala.thrift.TSlotRef;
 
 import com.google.common.base.Joiner;
-import com.google.common.base.Objects;
 import com.google.common.base.MoreObjects;
+import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
 
 public class SlotRef extends Expr {
@@ -253,6 +253,13 @@ public class SlotRef extends Expr {
   }
 
   @Override
+  public boolean referencesTuple(TupleId tid) {
+    Preconditions.checkState(type_.isValid());
+    Preconditions.checkState(desc_ != null);
+    return desc_.getParent().getId() == tid;
+  }
+
+  @Override
   public Expr clone() { return new SlotRef(this); }
 
   @Override
diff --git a/fe/src/main/java/org/apache/impala/analysis/SortInfo.java b/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
index 8821406..8a7780f 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SortInfo.java
@@ -266,9 +266,19 @@ public class SortInfo {
    * operator and 'offset' is the value in the 'OFFSET [x]' clause.
    */
   public long estimateTopNMaterializedSize(long cardinality, long offset) {
+    long totalRows = PlanNode.checkedAdd(cardinality, offset);
+    return estimateMaterializedSize(totalRows);
+  }
+
+  /**
+   * Estimates the size of 'totalRows' rows for this sort materialized in memory.
+   * The method uses the formula <code>estimatedSize = estimated # of rows in memory *
+   * average tuple serialized size</code>.
+   */
+  public long estimateMaterializedSize(long totalRows) {
     getSortTupleDescriptor().computeMemLayout();
     return (long) Math.ceil(getSortTupleDescriptor().getAvgSerializedSize()
-        * (PlanNode.checkedAdd(cardinality, offset)));
+        * totalRows);
   }
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
index 7f8b0b5..31463cf 100644
--- a/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/AnalyticEvalNode.java
@@ -410,13 +410,16 @@ public class AnalyticEvalNode extends PlanNode {
   public LimitPushdownInfo checkForLimitPushdown(SortInfo sortInfo,
           ExprSubstitutionMap sortInputSmap, SelectNode selectNode, long limit,
           SortNode analyticNodeSort, Analyzer analyzer) {
+    // Pushing is only supported for certain types of sort.
+    if (!analyticNodeSort.isPartitionedTopN() && !analyticNodeSort.isTotalSort()) {
+      return null;
+    }
     if (analyticFnCalls_.size() != 1) return null;
     Expr expr = analyticFnCalls_.get(0);
     if (!(expr instanceof FunctionCallExpr) ||
          (!AnalyticExpr.isRankingFn(((FunctionCallExpr) expr).getFn()))) {
       return null;
     }
-
     List<Expr> analyticSortSortExprs = analyticNodeSort.getSortInfo().getSortExprs();
 
     // In the mapping below, we use the original sort exprs that the sortInfo was
@@ -431,19 +434,7 @@ public class AnalyticEvalNode extends PlanNode {
     List<Expr> pbExprs = substitutedPartitionExprs_;
 
     if (sortExprs.size() == 0) {
-      // if there is no sort expr in the parent sort but only limit, we can push
-      // the limit to the sort below if there is no selection node or if
-      // the predicate in the selection node is eligible
-      if (selectNode == null) {
-        return new LimitPushdownInfo(false, 0);
-      }
-      Pair<LimitPushdownInfo, Double> status =
-              checkPredEligibleForLimitPushdown(selectNode.getConjuncts(), limit);
-      if (status.first != null) {
-        selectNode.setSelectivity(status.second);
-        return status.first;
-      }
-      return null;
+      return checkForUnorderedLimitPushdown(selectNode, limit, analyticNodeSort);
     }
 
     Preconditions.checkArgument(analyticSortSortExprs.size() >= pbExprs.size());
@@ -463,17 +454,21 @@ public class AnalyticEvalNode extends PlanNode {
     }
 
     if (selectNode == null) {
-      // Limit pushdown is valid if the pre-analytic sort puts rows into the same order
-      // as sortExprs. We check the prefix match, since extra analytic sort exprs do not
-      // affect the compatibility of the ordering with sortExprs.
+      // Do not support pushing additional limit to partitioned top-n, unless it was
+      // the predicate that the partitioned top-n was originally created from.
+      if (analyticNodeSort.isPartitionedTopN()) return null;
+      Preconditions.checkState(analyticNodeSort.isTotalSort());
+      // Limit pushdown to total sort is valid if the pre-analytic sort puts rows into
+      // the same order as sortExprs. We check the prefix match, since extra analytic
+      // sort exprs do not affect the compatibility of the ordering with sortExprs.
       if (!analyticSortExprsArePrefix(sortInfo, sortExprs,
               analyticNodeSort.getSortInfo(), analyticSortSortExprs)) {
         return null;
       }
       return new LimitPushdownInfo(false, 0);
     } else {
-      Pair<LimitPushdownInfo, Double> status =
-              checkPredEligibleForLimitPushdown(selectNode.getConjuncts(), limit);
+      Pair<LimitPushdownInfo, Double> status = checkPredEligibleForLimitPushdown(
+              analyticNodeSort, selectNode.getConjuncts(), limit);
       if (status.first != null) {
         selectNode.setSelectivity(status.second);
         return status.first;
@@ -483,6 +478,30 @@ public class AnalyticEvalNode extends PlanNode {
   }
 
   /**
+   * Helper for {@link #checkForLimitPushdown()} that handles the case of pushing
+   * down a limit into an total sort or partitioned top N where there are no
+   * ordering expressions.
+   */
+  private LimitPushdownInfo checkForUnorderedLimitPushdown(SelectNode selectNode,
+          long limit, SortNode analyticNodeSort) {
+    // Only support transforming total sort for now.
+    if (!analyticNodeSort.isTotalSort()) return null;
+    // if there is no sort expr in the parent sort but only limit, we can push
+    // the limit to the sort below if there is no selection node or if
+    // the predicate in the selection node is eligible
+    if (selectNode == null) {
+      return new LimitPushdownInfo(false, 0);
+    }
+    Pair<LimitPushdownInfo, Double> status = checkPredEligibleForLimitPushdown(
+            analyticNodeSort, selectNode.getConjuncts(), limit);
+    if (status.first != null) {
+      selectNode.setSelectivity(status.second);
+      return status.first;
+    }
+    return null;
+  }
+
+  /**
    * Checks if 'analyticSortExprs' is a prefix of the 'sortExprs' from an outer sort.
    * @param sortInfo sort info from the outer sort
    * @param sortExprs sort exprs from the outer sort. Must be a prefix of the full
@@ -551,7 +570,7 @@ public class AnalyticEvalNode extends PlanNode {
    *   null otherwise. Second value is the predicate's estimated selectivity
    */
   private Pair<LimitPushdownInfo, Double> checkPredEligibleForLimitPushdown(
-          List<Expr> conjuncts, long limit) {
+          SortNode analyticNodeSort, List<Expr> conjuncts, long limit) {
     Pair<LimitPushdownInfo, Double> falseStatus = new Pair<>(null, -1.0);
     // Currently, single conjuncts are supported.  In the future, multiple conjuncts
     // involving a range e.g 'col >= 10 AND col <= 20' could potentially be supported
@@ -610,12 +629,17 @@ public class AnalyticEvalNode extends PlanNode {
     // each analytic partition, so it is not safe to push the limit down.
     if (pred.getOp() == BinaryPredicate.Operator.EQ && limit > 1) return falseStatus;
 
-    // See method comment for explanation of why the analytic partition limit
-    // must be >= the pushed down limit.
-    if (analyticLimit < limit) return falseStatus;
+    // Check that the partition predicate matches the partition limits already pushed
+    // to a partitioned top-n node.
+    if (analyticNodeSort.isPartitionedTopN() &&
+            (analyticLimit != analyticNodeSort.getPerPartitionLimit() ||
+             includeTies != analyticNodeSort.isIncludeTies())) {
+      return falseStatus;
+    }
 
-    // Avoid overflow of limit.
-    if (analyticLimit + (long)limit > Integer.MAX_VALUE) return falseStatus;
+    LimitPushdownInfo pushdownInfo = checkLimitEligibleForPushdown(
+            limit, includeTies, analyticLimit);
+    if (pushdownInfo == null) return falseStatus;
 
     double selectivity = Expr.DEFAULT_SELECTIVITY;
     // Since the predicate is qualified for limit pushdown, estimate its selectivity.
@@ -625,8 +649,26 @@ public class AnalyticEvalNode extends PlanNode {
             pred.getOp() == BinaryPredicate.Operator.LE) {
       selectivity = 1.0;
     }
-    return new Pair<LimitPushdownInfo, Double>(
-            new LimitPushdownInfo(includeTies, (int)analyticLimit), selectivity);
+    return new Pair<LimitPushdownInfo, Double>(pushdownInfo, selectivity);
+  }
+
+  /**
+   * Implements the same check described in
+   * {@link #checkLimitEligibleForPushdown(long, boolean, long)} where we
+   * see if 'limit' can be pushed down into the pre-analytic sort where the
+   * there is a per-partition limit for the analytic 'analyticLimit', i.e.
+   * a row_number() < 10 predicate or similar.
+   */
+  private LimitPushdownInfo checkLimitEligibleForPushdown(long limit,
+          boolean includeTies, long analyticLimit) {
+    // See method comment for explanation of why the analytic partition limit
+    // must be >= the pushed down limit.
+    if (analyticLimit < limit) return null;
+
+    // Avoid overflow of limit.
+    if (analyticLimit + (long)limit > Integer.MAX_VALUE) return null;
+
+    return new LimitPushdownInfo(includeTies, (int)analyticLimit);
   }
 
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java b/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
index 57e38be..b15888a 100644
--- a/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/AnalyticPlanner.java
@@ -17,25 +17,31 @@
 
 package org.apache.impala.planner;
 
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
 import java.util.List;
 
+import org.apache.curator.shaded.com.google.common.collect.Iterables;
 import org.apache.impala.analysis.AggregateInfoBase;
 import org.apache.impala.analysis.AnalyticExpr;
 import org.apache.impala.analysis.AnalyticInfo;
 import org.apache.impala.analysis.AnalyticWindow;
 import org.apache.impala.analysis.Analyzer;
+import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.ExprSubstitutionMap;
+import org.apache.impala.analysis.NumericLiteral;
 import org.apache.impala.analysis.OrderByElement;
 import org.apache.impala.analysis.SlotDescriptor;
 import org.apache.impala.analysis.SlotRef;
 import org.apache.impala.analysis.SortInfo;
+import org.apache.impala.analysis.ToSqlOptions;
 import org.apache.impala.analysis.TupleDescriptor;
 import org.apache.impala.analysis.TupleId;
 import org.apache.impala.analysis.TupleIsNullPredicate;
+import org.apache.impala.catalog.Function;
 import org.apache.impala.common.ImpalaException;
 import org.apache.impala.thrift.TSortingOrder;
 import org.slf4j.Logger;
@@ -85,6 +91,8 @@ public class AnalyticPlanner {
    * 'groupingExprs'; if this is non-null, it returns in 'inputPartitionExprs'
    * a subset of the grouping exprs which should be used for the aggregate
    * hash partitioning during the parallelization of 'root'.
+   * Any unassigned conjuncts from 'analyzer_' are applied after analytic functions are
+   * evaluated.
    * TODO: when generating sort orders for the sort groups, optimize the ordering
    * of the partition exprs (so that subsequent sort operations see the input sorted
    * on a prefix of their required sort exprs)
@@ -93,6 +101,18 @@ public class AnalyticPlanner {
    */
   public PlanNode createSingleNodePlan(PlanNode root,
       List<Expr> groupingExprs, List<Expr> inputPartitionExprs) throws ImpalaException {
+    // Identify predicates that reference the logical analytic tuple (this logical
+    // analytic tuple is replaced by different physical ones during planning)
+    List<TupleId> tids = new ArrayList<>();
+    tids.addAll(root.getTupleIds());
+    tids.add(analyticInfo_.getOutputTupleId());
+    List<Expr> analyticConjs = analyzer_.getUnassignedConjuncts(tids);
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Analytic conjuncts: " +
+            Expr.listToSql(analyticConjs, ToSqlOptions.SHOW_IMPLICIT_CASTS));
+    }
+
+    List<PartitionLimit> perPartitionLimits = inferPartitionLimits(analyticConjs);
     List<WindowGroup> windowGroups = collectWindowGroups();
     for (int i = 0; i < windowGroups.size(); ++i) {
       windowGroups.get(i).init(analyzer_, "wg-" + i);
@@ -111,13 +131,43 @@ public class AnalyticPlanner {
           partitionGroups, groupingExprs, root.getNumInstances(), inputPartitionExprs);
     }
 
-    for (PartitionGroup partitionGroup: partitionGroups) {
-      for (int i = 0; i < partitionGroup.sortGroups.size(); ++i) {
-        root = createSortGroupPlan(root, partitionGroup.sortGroups.get(i),
-            i == 0 ? partitionGroup.partitionByExprs : null);
+    for (int i = 0; i < partitionGroups.size(); ++i) {
+      PartitionGroup partitionGroup = partitionGroups.get(i);
+      for (int j = 0; j < partitionGroup.sortGroups.size(); ++j) {
+        boolean lastSortGroup = (i == partitionGroups.size() - 1) &&
+                (j == partitionGroup.sortGroups.size() - 1);
+        root = createSortGroupPlan(root, partitionGroup.sortGroups.get(j),
+            j == 0 ? partitionGroup.partitionByExprs : null,
+            lastSortGroup ? perPartitionLimits : null);
+      }
+    }
+
+    List<Expr> substAnalyticConjs =
+        Expr.substituteList(analyticConjs, root.getOutputSmap(), analyzer_, false);
+    overrideSelectivityPushedLimits(analyticConjs, perPartitionLimits,
+            substAnalyticConjs);
+    return root.addConjunctsToNode(ctx_, analyzer_, tids, substAnalyticConjs);
+  }
+
+  /**
+   * Update selectivity of conjuncts in 'substAnalyticConjs' to reflect those that
+   * were pushed to a partitioned top-n.
+   *
+   * 'analyticConjs' and 'substAnalyticConjs' are the original conjuncts (matching
+   * the conjuncts in 'perPartitionLimits' and the substituted conjuncts and
+   * correspond to each other one to one.
+   */
+  private void overrideSelectivityPushedLimits(List<Expr> analyticConjs,
+          List<PartitionLimit> perPartitionLimits, List<Expr> substAnalyticConjs) {
+    for (PartitionLimit limit : perPartitionLimits) {
+      if (limit.pushed && limit.isLessThan) {
+        int idx = analyticConjs.indexOf(limit.conjunct);
+        if (idx >= 0) {
+          substAnalyticConjs.set(idx,
+                  substAnalyticConjs.get(idx).cloneAndOverrideSelectivity(1.0));
+        }
       }
     }
-    return root;
   }
 
   /**
@@ -328,9 +378,13 @@ public class AnalyticPlanner {
    * Marks the SortNode as requiring its input to be partitioned if partitionExprs
    * is not null (partitionExprs represent the data partition of the entire partition
    * group of which this sort group is a part).
+   *
+   * If 'perPartitionLimits' is non-null, attempt to place these limits in the sort.
+   * Any placed limits have 'pushed' set to true.
    */
   private PlanNode createSortGroupPlan(PlanNode root, SortGroup sortGroup,
-      List<Expr> partitionExprs) throws ImpalaException {
+      List<Expr> partitionExprs, List<PartitionLimit> perPartitionLimits)
+              throws ImpalaException {
     List<Expr> partitionByExprs = sortGroup.partitionByExprs;
     List<OrderByElement> orderByElements = sortGroup.orderByElements;
     boolean hasActivePartition = !Expr.allConstant(partitionByExprs);
@@ -367,8 +421,51 @@ public class AnalyticPlanner {
       SortInfo sortInfo = createSortInfo(root, sortExprs, isAsc, nullsFirst);
       // IMPALA-8533: Avoid generating sort with empty tuple descriptor
       if(sortInfo.getSortTupleDescriptor().getSlots().size() > 0) {
-        sortNode =
-            SortNode.createTotalSortNode(ctx_.getNextNodeId(), root, sortInfo, 0);
+        // Select the lowest limit to push into the sort. Other limit conjuncts will
+        // be added later, if needed. We could try to merge limits together when one
+        // dominates the other, e.g. RANK() < 10 and RANK() < 20. However, for
+        // simplicity, and in particular to avoid the need to handle cases where neither
+        // dominates the other, e.g. ROW_NUMBER() < 10 and RANK() < 20, we only pick one
+        // limit. Remaining limits will be assigned as predicates later in analytic
+        // planning.
+        PartitionLimit limit = null;
+        if (perPartitionLimits != null && sortGroup.windowGroups.size() == 1) {
+          for (PartitionLimit p : perPartitionLimits) {
+            if (sortGroup.windowGroups.get(0).analyticExprs.contains(p.analyticExpr)) {
+              if (limit == null || p.limit < limit.limit) {
+                limit = p;
+              }
+            }
+          }
+        }
+
+        if (limit == null ||
+            limit.limit > analyzer_.getQueryOptions().analytic_rank_pushdown_threshold) {
+          // Generate a full sort if no limit is known, or if the limit is large enough
+          // to disable rank pushed. Even if a limit is known, the full sort can be more
+          // efficient the the in-memory top-n if the limit is large enough.
+          sortNode = SortNode.createTotalSortNode(
+               ctx_.getNextNodeId(), root, sortInfo, 0);
+        } else {
+          // The backend can't handle limits < 1. We need to apply a limit of 1, and
+          // evaluate the predicate later.
+          // TODO: IMPALA-10015: we should instead generate an empty set plan that
+          // produces the appropriate tuples. This would require short-circuiting plan
+          // generation to avoid generating the whole select plan.
+          long planNodeLimit = Math.max(1, limit.limit);
+          // Convert to standard top-N if only one partition.
+          if (Iterables.all(partitionByExprs, Expr.IS_LITERAL_VALUE)) {
+            sortNode = SortNode.createTopNSortNode(ctx_.getQueryOptions(),
+                    ctx_.getNextNodeId(), root, sortInfo, 0, planNodeLimit,
+                    limit.includeTies);
+          } else {
+            sortNode = SortNode.createPartitionedTopNSortNode(
+                    ctx_.getNextNodeId(), root, sortInfo, partitionByExprs.size(),
+                    planNodeLimit, limit.includeTies);
+          }
+          sortNode.setLimitSrcPred(limit.conjunct);
+          limit.markPushed();
+        }
 
         // if this sort group does not have partitioning exprs, we want the sort
         // to be executed like a regular distributed sort
@@ -382,7 +479,6 @@ public class AnalyticPlanner {
           }
           sortNode.setInputPartition(inputPartition);
         }
-
         root = sortNode;
         root.init(analyzer_);
       }
@@ -758,4 +854,126 @@ public class AnalyticPlanner {
     }
     return partitionGroups;
   }
+
+  private static class PartitionLimit {
+    public PartitionLimit(Expr conjunct, AnalyticExpr analyticExpr, long limit,
+            boolean includeTies, boolean isLessThan) {
+      this.conjunct = conjunct;
+      this.analyticExpr = analyticExpr;
+      this.limit = limit;
+      this.includeTies = includeTies;
+      this.isLessThan = isLessThan;
+      this.pushed = false;
+    }
+
+    /// The conjunct that this was derived from.
+    public final Expr conjunct;
+
+    /// The ranking analytic expr that this limit was inferred from.
+    public final AnalyticExpr analyticExpr;
+
+    /// The limit on rows per partition returned.
+    public final long limit;
+
+    /// Whether ties for last place need to be included.
+    public final boolean includeTies;
+
+    /// Whether the source predicate was a simple < or <= inequality.
+    /// I.e. false for = and true for < and <=.
+    public final boolean isLessThan;
+
+    /// Whether the limit was pushed to a top-n operator.
+    private boolean pushed;
+
+    public boolean isPushed() {
+      return pushed;
+    }
+
+    public void markPushed() {
+      this.pushed = true;
+    }
+  }
+
+  /**
+   * Extract per-partition limits from 'conjuncts'.
+   */
+  private List<PartitionLimit> inferPartitionLimits(List<Expr> conjuncts) {
+    List<PartitionLimit> result = new ArrayList<>();
+    if (analyzer_.getQueryOptions().analytic_rank_pushdown_threshold <= 0) return result;
+    for (Expr conj : conjuncts) {
+      if (!(Expr.IS_BINARY_PREDICATE.apply(conj))) continue;
+      BinaryPredicate pred = (BinaryPredicate) conj;
+      Expr lhs = pred.getChild(0);
+      Expr rhs = pred.getChild(1);
+      // Lhs of the binary predicate must be a ranking function.
+      // Also, it must be bound to the output tuple of this analytic eval node
+      if (!(lhs instanceof SlotRef)) continue;
+
+      List<Expr> lhsSourceExprs = ((SlotRef) lhs).getDesc().getSourceExprs();
+      if (lhsSourceExprs.size() > 1 ||
+            !(lhsSourceExprs.get(0) instanceof AnalyticExpr)) {
+        continue;
+      }
+
+      boolean includeTies;
+      AnalyticExpr analyticExpr = (AnalyticExpr) lhsSourceExprs.get(0);
+      Function fn = analyticExpr.getFnCall().getFn();
+      // Check if this a predicate that we can convert to a limit in the sort.
+      // We do not have the runtime support that would be required to handle
+      // DENSE_RANK().
+      if (AnalyticExpr.isAnalyticFn(fn, AnalyticExpr.RANK)) {
+        // RANK() assigns equal values to rows where the ORDER BY expressions are equal.
+        // Thus if there are ties for the max RANK() value to be returned, we need to
+        // return all of them. E.g. if the predicate is RANK() <= 3, and the values we are
+        // ordering by are 1, 42, 99, 99, 100, then the RANK() values are 1, 2, 3, 3, 5
+        // and we need to return the top 4 rows: 1, 42, 99, 99.
+        // We do not need special handling for ties except at the limit value. E.g. if
+        // the ordering values were 1, 1, 42, 99, 100, then the RANK() values would be
+        // 1, 1, 3, 4, 5 and we only return the top 3.
+        includeTies = true;
+      } else if (AnalyticExpr.isAnalyticFn(fn, AnalyticExpr.ROWNUMBER)) {
+        includeTies = false;
+      } else {
+        continue;
+      }
+
+      AnalyticWindow window = analyticExpr.getWindow();
+      // Check that the window frame is UNBOUNDED PRECEDING to CURRENT ROW,
+      // i.e. that the function monotonically increases from 1 within the window.
+      if (window.getLeftBoundary().getType() !=
+            AnalyticWindow.BoundaryType.UNBOUNDED_PRECEDING
+          || window.getRightBoundary().getType() !=
+              AnalyticWindow.BoundaryType.CURRENT_ROW) {
+        continue;
+      }
+
+      // Next, try to extract a numeric limit that will apply to the analytic function.
+      if (!(rhs instanceof NumericLiteral)) continue;
+      BigDecimal val = ((NumericLiteral)rhs).getValue();
+      if (val.compareTo(new BigDecimal(Long.MAX_VALUE)) > 0) {
+        continue;
+      }
+      // Round down to the nearest long.
+      long longVal = val.longValue();
+      long perPartitionLimit;
+      // currently, restrict the pushdown for =, <, <= predicates
+      if (pred.getOp() == BinaryPredicate.Operator.EQ ||
+              pred.getOp() == BinaryPredicate.Operator.LE) {
+        perPartitionLimit = longVal;
+      } else if (pred.getOp() == BinaryPredicate.Operator.LT) {
+        perPartitionLimit = longVal - 1;
+      } else {
+        continue;
+      }
+      boolean isLessThan = pred.getOp() == BinaryPredicate.Operator.LE ||
+              pred.getOp() == BinaryPredicate.Operator.LT;
+      if (LOG.isTraceEnabled()) {
+        LOG.trace(analyticExpr.debugString() + " implies per-partition limit " +
+             perPartitionLimit + " includeTies=" + includeTies);
+      }
+      result.add(new PartitionLimit(
+              conj, analyticExpr, perPartitionLimit, includeTies, isLessThan));
+    }
+    return result;
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java b/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
index 3867d21..8a75f9d 100644
--- a/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/DistributedPlanner.java
@@ -1086,7 +1086,7 @@ public class DistributedPlanner {
       // Make sure the childFragment's output is partitioned as required by the sortNode.
       DataPartition sortPartition = sortNode.getInputPartition();
       if (!childFragment.getDataPartition().equals(sortPartition)) {
-        if (sortNode.isTypeTopN()) {
+        if (sortNode.isTypeTopN() || sortNode.isPartitionedTopN()) {
           lowerTopN = sortNode;
           childFragment.addPlanRoot(lowerTopN);
           // Update partitioning exprs to reference sort tuple.
@@ -1106,10 +1106,18 @@ public class DistributedPlanner {
     }
     if (addedLowerTopN) {
       // Create the upper TopN node
-      SortNode upperTopN = SortNode.createTopNSortNode(ctx_.getQueryOptions(),
+      SortNode upperTopN;
+      if (lowerTopN.isTypeTopN()) {
+        upperTopN = SortNode.createTopNSortNode(ctx_.getQueryOptions(),
               ctx_.getNextNodeId(), childFragment.getPlanRoot(),
               lowerTopN.getSortInfo(), sortNode.getOffset(), lowerTopN.getSortLimit(),
               lowerTopN.isIncludeTies());
+      } else {
+        upperTopN = SortNode.createPartitionedTopNSortNode(
+                ctx_.getNextNodeId(), childFragment.getPlanRoot(),
+                lowerTopN.getSortInfo(), lowerTopN.getNumPartitionExprs(),
+                lowerTopN.getPerPartitionLimit(), lowerTopN.isIncludeTies());
+      }
       upperTopN.setIsAnalyticSort(true);
       upperTopN.init(ctx_.getRootAnalyzer());
       // connect this to the analytic eval node
@@ -1143,15 +1151,28 @@ public class DistributedPlanner {
     if (node.isIncludeTies()) {
       Preconditions.checkState(node.getOffset() == 0,
               "Tie handling with offset not supported");
-      // TopN that returns ties needs special handling because ties are not handled
-      // correctly by the ExchangeNode limit. We need to generate a top-n on top
-      // of the exchange to correctly merge the input.
-      SortNode parentSortNode = SortNode.createTopNSortNode(
-              ctx_.getQueryOptions(), ctx_.getNextNodeId(), exchNode,
-              childSortNode.getSortInfo(), 0, node.getSortLimit(),
-              childSortNode.isIncludeTies());
-      parentSortNode.init(ctx_.getRootAnalyzer());
-      mergeFragment.addPlanRoot(parentSortNode);
+      if (node.isPartitionedTopN()) {
+        // Partitioned TopN that returns ties needs special handling because ties are not
+        // handled correctly by the ExchangeNode limit. We need to generate a partitioned
+        // top-n on top of the exchange to correctly merge the input.
+        SortNode parentSortNode = SortNode.createPartitionedTopNSortNode(
+                ctx_.getNextNodeId(), exchNode, childSortNode.getSortInfo(),
+                childSortNode.getNumPartitionExprs(),
+                childSortNode.getPerPartitionLimit(), childSortNode.isIncludeTies());
+        parentSortNode.init(ctx_.getRootAnalyzer());
+        mergeFragment.addPlanRoot(parentSortNode);
+      } else {
+        Preconditions.checkState(node.isTypeTopN(), "only top-n handles ties");
+        //TopN that returns ties needs special handling because ties are not handled
+        // correctly by the ExchangeNode limit. We need to generate a top-n on top
+        // of the exchange to correctly merge the input.
+        SortNode parentSortNode = SortNode.createTopNSortNode(
+                ctx_.getQueryOptions(), ctx_.getNextNodeId(), exchNode,
+                childSortNode.getSortInfo(), 0, node.getSortLimit(),
+                childSortNode.isIncludeTies());
+        parentSortNode.init(ctx_.getRootAnalyzer());
+        mergeFragment.addPlanRoot(parentSortNode);
+      }
     } else {
       // Remember original offset and limit.
       boolean hasLimit = node.hasLimit();
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanNode.java b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
index f0b2c09..a91cb4d 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlanNode.java
@@ -26,9 +26,11 @@ import java.util.List;
 import java.util.Set;
 
 import org.apache.impala.analysis.Analyzer;
+import org.apache.impala.analysis.BinaryPredicate;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.analysis.ExprId;
 import org.apache.impala.analysis.ExprSubstitutionMap;
+import org.apache.impala.analysis.SlotDescriptor;
 import org.apache.impala.analysis.ToSqlOptions;
 import org.apache.impala.analysis.TupleDescriptor;
 import org.apache.impala.analysis.TupleId;
@@ -517,6 +519,44 @@ abstract public class PlanNode extends TreeNode<PlanNode> {
   }
 
   /**
+   * Apply the provided conjuncts to the this node, returning the new root of
+   * the plan tree. Also add any slot equivalences for tupleIds that have not
+   * yet been enforced.
+   * TODO: change this to assign the unassigned conjuncts to root itself, if that is
+   * semantically correct
+   */
+  public PlanNode addConjunctsToNode(PlannerContext plannerCtx,
+          Analyzer analyzer, List<TupleId> tupleIds, List<Expr> conjuncts) {
+    if (LOG.isTraceEnabled()) {
+      LOG.trace(String.format("conjuncts for (Node %s): %s",
+          getDisplayLabel(), Expr.debugString(conjuncts)));
+      LOG.trace("all conjuncts: " + analyzer.conjunctAssignmentsDebugString());
+    }
+    if (this instanceof EmptySetNode) return this;
+    for (TupleId tid: tupleIds) {
+      analyzer.createEquivConjuncts(tid, conjuncts);
+    }
+    if (conjuncts.isEmpty()) return this;
+
+    List<Expr> finalConjuncts = new ArrayList<>();
+    // Check if this is an inferred identity predicate i.e for c1 = c2 both
+    // sides are pointing to the same source slot. In such cases it is wrong
+    // to add the predicate to the SELECT node because it will incorrectly
+    // eliminate rows with NULL values.
+    for (Expr e : conjuncts) {
+      if (e instanceof BinaryPredicate && ((BinaryPredicate) e).isInferred()) {
+        SlotDescriptor lhs = ((BinaryPredicate) e).getChild(0).findSrcScanSlot();
+        SlotDescriptor rhs = ((BinaryPredicate) e).getChild(1).findSrcScanSlot();
+        if (lhs != null && rhs != null && lhs.equals(rhs)) continue;
+      }
+      finalConjuncts.add(e);
+    }
+    if (finalConjuncts.isEmpty()) return this;
+    // The conjuncts need to be evaluated in a SelectNode.
+    return SelectNode.create(plannerCtx, analyzer, this, finalConjuncts);
+  }
+
+  /**
    * Returns an smap that combines the childrens' smaps.
    */
   protected ExprSubstitutionMap getCombinedChildSmap() {
diff --git a/fe/src/main/java/org/apache/impala/planner/SelectNode.java b/fe/src/main/java/org/apache/impala/planner/SelectNode.java
index 48d033a..fa888f4 100644
--- a/fe/src/main/java/org/apache/impala/planner/SelectNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SelectNode.java
@@ -19,15 +19,14 @@ package org.apache.impala.planner;
 
 import java.util.List;
 
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import org.apache.impala.analysis.Analyzer;
 import org.apache.impala.analysis.Expr;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
 import org.apache.impala.thrift.TPlanNodeType;
 import org.apache.impala.thrift.TQueryOptions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.base.Preconditions;
 
@@ -68,6 +67,31 @@ public class SelectNode extends PlanNode {
     createDefaultSmap(analyzer);
   }
 
+  /**
+   * Create a SelectNode that evaluates 'conjuncts' on output rows from 'root',
+   * or merge 'conjuncts' into 'root' if it is already a SelectNode.
+   */
+  public static PlanNode create(PlannerContext plannerCtx, Analyzer analyzer,
+          PlanNode root, List<Expr> conjuncts) {
+    SelectNode selectNode;
+    if (root instanceof SelectNode && !root.hasLimit()) {
+      selectNode = (SelectNode) root;
+      // This is a select node that evaluates conjuncts only. We can
+      // safely merge conjuncts from the child SelectNode into this one.
+      for (Expr conjunct : conjuncts) {
+        if (!selectNode.conjuncts_.contains(conjunct)) {
+          selectNode.conjuncts_.add(conjunct);
+        }
+      }
+    } else {
+      selectNode = new SelectNode(plannerCtx.getNextNodeId(), root, conjuncts);
+    }
+    // init() marks conjuncts as assigned
+    selectNode.init(analyzer);
+    Preconditions.checkState(selectNode.hasValidStats());
+    return selectNode;
+  }
+
   @Override
   public void computeStats(Analyzer analyzer) {
     super.computeStats(analyzer);
diff --git a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
index a556d80..b004fae 100644
--- a/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
+++ b/fe/src/main/java/org/apache/impala/planner/SingleNodePlanner.java
@@ -289,6 +289,7 @@ public class SingleNodePlanner {
         } else {
           groupingExprs = Collections.emptyList();
         }
+
         List<Expr> inputPartitionExprs = new ArrayList<>();
         root = analyticPlanner.createSingleNodePlan(
             root, groupingExprs, inputPartitionExprs);
@@ -436,10 +437,8 @@ public class SingleNodePlanner {
 
   /**
    * If there are unassigned conjuncts that are bound by tupleIds or if there are slot
-   * equivalences for tupleIds that have not yet been enforced, returns a SelectNode on
-   * top of root that evaluates those conjuncts; otherwise returns root unchanged.
-   * TODO: change this to assign the unassigned conjuncts to root itself, if that is
-   * semantically correct
+   * equivalences for tupleIds that have not yet been enforced, add them to the plan
+   * tree, returning either the original root or a new root.
    */
   private PlanNode addUnassignedConjuncts(
       Analyzer analyzer, List<TupleId> tupleIds, PlanNode root) {
@@ -449,37 +448,7 @@ public class SingleNodePlanner {
     // Gather unassigned conjuncts and generate predicates to enforce
     // slot equivalences for each tuple id.
     List<Expr> conjuncts = analyzer.getUnassignedConjuncts(root);
-    if (LOG.isTraceEnabled()) {
-      LOG.trace(String.format("unassigned conjuncts for (Node %s): %s",
-          root.getDisplayLabel(), Expr.debugString(conjuncts)));
-      LOG.trace("all conjuncts: " + analyzer.conjunctAssignmentsDebugString());
-    }
-    for (TupleId tid: tupleIds) {
-      analyzer.createEquivConjuncts(tid, conjuncts);
-    }
-    if (conjuncts.isEmpty()) return root;
-
-    List<Expr> finalConjuncts = new ArrayList<>();
-    // Check if this is an inferred identity predicate i.e for c1 = c2 both
-    // sides are pointing to the same source slot. In such cases it is wrong
-    // to add the predicate to the SELECT node because it will incorrectly
-    // eliminate rows with NULL values.
-    for (Expr e : conjuncts) {
-      if (e instanceof BinaryPredicate && ((BinaryPredicate) e).isInferred()) {
-        SlotDescriptor lhs = ((BinaryPredicate) e).getChild(0).findSrcScanSlot();
-        SlotDescriptor rhs = ((BinaryPredicate) e).getChild(1).findSrcScanSlot();
-        if (lhs != null && rhs != null && lhs.equals(rhs)) continue;
-      }
-      finalConjuncts.add(e);
-    }
-    if (finalConjuncts.isEmpty()) return root;
-
-    // evaluate conjuncts in SelectNode
-    SelectNode selectNode = new SelectNode(ctx_.getNextNodeId(), root, finalConjuncts);
-    // init() marks conjuncts as assigned
-    selectNode.init(analyzer);
-    Preconditions.checkState(selectNode.hasValidStats());
-    return selectNode;
+    return root.addConjunctsToNode(ctx_, analyzer, tupleIds, conjuncts);
   }
 
   /**
@@ -1345,7 +1314,7 @@ public class SingleNodePlanner {
    * makes the *output* of the computation visible to the enclosing scope, so that
    * filters from the enclosing scope can be safely applied (to the grouping cols, say).
    */
-  public void migrateConjunctsToInlineView(final Analyzer analyzer,
+  private void migrateConjunctsToInlineView(final Analyzer analyzer,
       final InlineViewRef inlineViewRef) throws ImpalaException {
     List<TupleId> tids = inlineViewRef.getId().asList();
     if (inlineViewRef.isTableMaskingView()
@@ -1354,9 +1323,23 @@ public class SingleNodePlanner {
     }
     List<Expr> unassignedConjuncts = analyzer.getUnassignedConjuncts(tids, true);
     if (LOG.isTraceEnabled()) {
-      LOG.trace("unassignedConjuncts: " + Expr.debugString(unassignedConjuncts));
+      LOG.trace("migrateConjunctsToInlineView() unassignedConjuncts: " +
+          Expr.debugString(unassignedConjuncts));
     }
     if (!canMigrateConjuncts(inlineViewRef)) {
+      // We may be able to migrate some specific analytic conjuncts into the view.
+      List<Expr> analyticPreds = findAnalyticConjunctsToMigrate(analyzer, inlineViewRef,
+              unassignedConjuncts);
+      if (analyticPreds.size() > 0) {
+        if (LOG.isTraceEnabled()) {
+          LOG.trace("Migrate analytic predicates into view " +
+              Expr.debugString(analyticPreds));
+        }
+        analyzer.markConjunctsAssigned(analyticPreds);
+        unassignedConjuncts.removeAll(analyticPreds);
+        addConjunctsIntoInlineView(analyzer, inlineViewRef, analyticPreds);
+      }
+
       // mark (fully resolve) slots referenced by unassigned conjuncts as
       // materialized
       List<Expr> substUnassigned = Expr.substituteList(unassignedConjuncts,
@@ -1377,6 +1360,53 @@ public class SingleNodePlanner {
     // Propagate the conjuncts evaluating the nullable side of outer-join.
     // Don't mark them as assigned so they would be assigned at the JOIN node.
     preds.addAll(evalAfterJoinPreds);
+    addConjunctsIntoInlineView(analyzer, inlineViewRef, preds);
+
+    // mark (fully resolve) slots referenced by remaining unassigned conjuncts as
+    // materialized
+    List<Expr> substUnassigned = Expr.substituteList(unassignedConjuncts,
+        inlineViewRef.getBaseTblSmap(), analyzer, false);
+    analyzer.materializeSlots(substUnassigned);
+  }
+
+  /**
+   * Return any conjuncts in 'conjuncts' that reference analytic exprs in 'inlineViewRef'
+   * and can be safely migrated into 'inlineViewRef', even if
+   * canMigrateConjuncts(inlineViewRef) is false.
+   */
+  private List<Expr> findAnalyticConjunctsToMigrate(final Analyzer analyzer,
+          final InlineViewRef inlineViewRef, List<Expr> conjuncts) {
+    // Not safe to migrate if the inline view has a limit or offset that is applied after
+    // analytic function evaluation.
+    if (inlineViewRef.getViewStmt().hasLimit() || inlineViewRef.getViewStmt().hasOffset()
+            || !(inlineViewRef.getViewStmt() instanceof SelectStmt)) {
+      return Collections.emptyList();
+    }
+    SelectStmt selectStmt = ((SelectStmt) inlineViewRef.getViewStmt());
+    if (!selectStmt.hasAnalyticInfo()) return Collections.emptyList();
+
+    // Find conjuncts that reference the (logical) analytic tuple. These conjuncts will
+    // only be evaluated after the analytic functions in the subquery so will not migrate
+    // to be evaluated earlier in the plan (which could produce incorrect results).
+    TupleDescriptor analyticTuple = selectStmt.getAnalyticInfo().getOutputTupleDesc();
+    List<Expr> analyticPreds = new ArrayList<>();
+    for (int i = 0; i < conjuncts.size(); ++i) {
+      Expr pred = conjuncts.get(i);
+      Expr viewPred = pred.substitute(inlineViewRef.getSmap(), analyzer, false);
+      if (viewPred.referencesTuple(analyticTuple.getId())) {
+        analyticPreds.add(pred);
+      }
+    }
+    return analyticPreds;
+  }
+
+  /**
+   * Add the provided conjuncts to be evaluated inside 'inlineViewRef'.
+   * Does not mark the conjuncts as assigned.
+   */
+  private void addConjunctsIntoInlineView(final Analyzer analyzer,
+          final InlineViewRef inlineViewRef, List<Expr> preds)
+                  throws AnalysisException {
     // Generate predicates to enforce equivalences among slots of the inline view
     // tuple. These predicates are also migrated into the inline view.
     analyzer.createEquivConjuncts(inlineViewRef.getId(), preds);
@@ -1428,12 +1458,6 @@ public class SingleNodePlanner {
     // apply to the post-join/agg/analytic result of the inline view.
     for (Expr e: viewPredicates) e.setIsOnClauseConjunct(false);
     inlineViewRef.getAnalyzer().registerConjuncts(viewPredicates);
-
-    // mark (fully resolve) slots referenced by remaining unassigned conjuncts as
-    // materialized
-    List<Expr> substUnassigned = Expr.substituteList(unassignedConjuncts,
-        inlineViewRef.getBaseTblSmap(), analyzer, false);
-    analyzer.materializeSlots(substUnassigned);
   }
 
   /**
diff --git a/fe/src/main/java/org/apache/impala/planner/SortNode.java b/fe/src/main/java/org/apache/impala/planner/SortNode.java
index bfcebd5..b0ee413 100644
--- a/fe/src/main/java/org/apache/impala/planner/SortNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/SortNode.java
@@ -26,6 +26,7 @@ import org.apache.impala.analysis.ExprSubstitutionMap;
 import org.apache.impala.analysis.SlotDescriptor;
 import org.apache.impala.analysis.SlotRef;
 import org.apache.impala.analysis.SortInfo;
+import org.apache.impala.analysis.ToSqlOptions;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.thrift.TExplainLevel;
 import org.apache.impala.thrift.TPlanNode;
@@ -75,14 +76,26 @@ public class SortNode extends PlanNode {
   // The offset of the first row to return.
   protected long offset_;
 
+  // How many of the expressions in info_ comprise the partition key.
+  // Non-negative if type_ is PARTITIONED_TOPN, -1 otherwise.
+  protected int numPartitionExprs_;
+
+  // Max rows to return for each partition key value.
+  // Non-negative if type_ is PARTITIONED_TOPN, -1 otherwise.
+  protected long perPartitionLimit_;
+
   // Whether to include ties for the last place in the Top-N values.
-  // Only supported if type_ is TOPN.
+  // Only supported if type_ is TOPN or PARTITIONED_TOPN.
   protected boolean includeTies_;
 
   // If includeTies_ is true, this is the limit used. We cannot use the default 'limit_'
   // because the plan node may return more than this number of rows.
   protected long limitWithTies_;
 
+  // The predicate that the limit was derived from, if any. If non-null,
+  // used for informational purposes in the explain string.
+  protected Expr limitSrcPred_;
+
   // The type of sort. Determines the exec node used in the BE.
   private TSortType type_;
 
@@ -95,7 +108,7 @@ public class SortNode extends PlanNode {
    */
   public static SortNode createPartialSortNode(
       PlanNodeId id, PlanNode input, SortInfo info) {
-    return new SortNode(id, input, info, 0, -1, false, TSortType.PARTIAL);
+    return new SortNode(id, input, info, 0, -1, -1, -1, false, TSortType.PARTIAL);
   }
 
   /**
@@ -113,7 +126,8 @@ public class SortNode extends PlanNode {
     SortNode result;
     if (topNBytesLimit <= 0 || estimatedTopNMaterializedSize < topNBytesLimit
             || includeTies) {
-      result = new SortNode(id, input, info, offset, limit, includeTies, TSortType.TOPN);
+      result = new SortNode(
+              id, input, info, offset, limit, -1, -1, includeTies, TSortType.TOPN);
     } else {
       result = SortNode.createTotalSortNode(id, input, info, offset);
       result.setLimit(limit);
@@ -122,24 +136,45 @@ public class SortNode extends PlanNode {
   }
 
   /**
+   * Creates a new SortNode with a per-partition limit that is executed with TopNNode
+   * in the BE.
+   */
+  public static SortNode createPartitionedTopNSortNode(
+      PlanNodeId id, PlanNode input, SortInfo info, int numPartitionExprs,
+      long perPartitionLimit, boolean includeTies) {
+    return new SortNode(id, input, info, 0, -1, numPartitionExprs, perPartitionLimit,
+        includeTies, TSortType.PARTITIONED_TOPN);
+  }
+
+  /**
    * Creates a new SortNode that does a total sort, possibly with a limit.
    */
   public static SortNode createTotalSortNode(
       PlanNodeId id, PlanNode input, SortInfo info, long offset) {
-    return new SortNode(id, input, info, offset, -1, false, TSortType.TOTAL);
+    return new SortNode(id, input, info, offset, -1, -1, -1, false, TSortType.TOTAL);
   }
 
   private SortNode(
       PlanNodeId id, PlanNode input, SortInfo info, long offset, long limit,
-      boolean includeTies, TSortType type) {
+      int numPartitionExprs, long perPartitionLimit, boolean includeTies,
+      TSortType type) {
     super(id, info.getSortTupleDescriptor().getId().asList(), getDisplayName(type));
     Preconditions.checkState(offset >= 0);
-    Preconditions.checkArgument(type != TSortType.TOPN || limit >= 0);
+    if (type == TSortType.PARTITIONED_TOPN) {
+      // We need to support 0 partition exprs if ties are included, because the
+      // non-partitioned Top-N and sort nodes do not currently support including ties.
+      Preconditions.checkState(includeTies || numPartitionExprs > 0);
+      Preconditions.checkState(perPartitionLimit > 0);
+    } else if (type == TSortType.TOPN) {
+      Preconditions.checkArgument(type != TSortType.TOPN || limit >= 0);
+    }
     info_ = info;
     children_.add(input);
     offset_ = offset;
+    numPartitionExprs_ = numPartitionExprs;
+    perPartitionLimit_ = perPartitionLimit;
     includeTies_ = includeTies;
-    limitWithTies_ = includeTies ? limit : -1;
+    limitWithTies_ = (type == TSortType.TOPN && includeTies) ? limit : -1;
     type_ = type;
     if (!includeTies) setLimit(limit);
   }
@@ -147,7 +182,11 @@ public class SortNode extends PlanNode {
   public long getOffset() { return offset_; }
   public void setOffset(long offset) { offset_ = offset; }
   public boolean hasOffset() { return offset_ > 0; }
+  public boolean isTotalSort() { return type_ == TSortType.TOTAL; }
   public boolean isTypeTopN() { return type_ == TSortType.TOPN; }
+  public boolean isPartitionedTopN() { return type_ == TSortType.PARTITIONED_TOPN; }
+  public int getNumPartitionExprs() { return numPartitionExprs_; }
+  public long getPerPartitionLimit() { return perPartitionLimit_; }
   public boolean isIncludeTies() { return includeTies_; }
   // Get the limit that applies to the sort, including ties or not.
   public long getSortLimit() {
@@ -165,12 +204,14 @@ public class SortNode extends PlanNode {
   public AnalyticEvalNode getAnalyticEvalNode() { return analyticEvalNode_; }
 
   /**
-   * Under special cases, the planner may decide to convert a total sort into a
-   * TopN sort with limit. This does the conversion to top-n if the converted
-   * node would pass the TOPN_BYTES_LIMIT check. Otherwise does not modify this node.
+   * Under special cases, the planner may decide to convert a total sort or
+   * partition top-N into a TopN sort with limit. This does the conversion to top-n
+   * if the converted node would pass the TOPN_BYTES_LIMIT check. Otherwise does
+   * not modify this node.
    */
   public void tryConvertToTopN(long limit, Analyzer analyzer, boolean includeTies) {
-    Preconditions.checkArgument(type_ == TSortType.TOTAL);
+    Preconditions.checkArgument(type_ == TSortType.TOTAL
+        || type_ == TSortType.PARTITIONED_TOPN);
     Preconditions.checkState(!hasLimit());
     Preconditions.checkState(!hasOffset());
     long topNBytesLimit = analyzer.getQueryOptions().topn_bytes_limit;
@@ -183,6 +224,8 @@ public class SortNode extends PlanNode {
     }
     type_ = TSortType.TOPN;
     displayName_ = getDisplayName(type_);
+    numPartitionExprs_ = -1;
+    perPartitionLimit_ = -1;
     includeTies_ = includeTies;
     if (includeTies) {
       unsetLimit();
@@ -199,6 +242,10 @@ public class SortNode extends PlanNode {
     return super.allowPartitioned();
   }
 
+  public void setLimitSrcPred(Expr v) {
+    this.limitSrcPred_ = v;
+  }
+
   @Override
   public boolean isBlockingNode() { return type_ != TSortType.PARTIAL; }
 
@@ -248,11 +295,23 @@ public class SortNode extends PlanNode {
   @Override
   protected void computeStats(Analyzer analyzer) {
     super.computeStats(analyzer);
-    if (includeTies_) {
+    if (isTypeTopN() && includeTies_) {
       cardinality_ = capCardinalityAtLimit(getChild(0).cardinality_, limitWithTies_);
     } else {
       cardinality_ = capCardinalityAtLimit(getChild(0).cardinality_);
     }
+    if (type_ == TSortType.PARTITIONED_TOPN) {
+      // We may be able to get a more precise estimate based on the number of
+      // partitions and per-partition limits.
+      List<Expr> partExprs = info_.getSortExprs().subList(0, numPartitionExprs_);
+      long partNdv = numPartitionExprs_ == 0 ? 1 : Expr.getNumDistinctValues(partExprs);
+      if (partNdv >= 0) {
+        long maxRowsInHeaps = partNdv * getPerPartitionLimit();
+        if (cardinality_ < 0 || cardinality_ > maxRowsInHeaps) {
+          cardinality_ = maxRowsInHeaps;
+        }
+      }
+    }
 
     if (LOG.isTraceEnabled()) {
       LOG.trace("stats Sort: cardinality=" + Long.toString(cardinality_));
@@ -273,17 +332,21 @@ public class SortNode extends PlanNode {
         .add("offset_", offset_)
         .add("includeTies_", includeTies_)
         .add("limitWithTies_", limitWithTies_)
+        .add("numPartitionExprs_", numPartitionExprs_)
+        .add("perPartitionLimit_", perPartitionLimit_)
         .addValue(super.debugString())
         .toString();
   }
 
   @Override
   protected void toThrift(TPlanNode msg) {
-    Preconditions.checkState(!isTypeTopN() || hasLimit() ||
-            (includeTies_ && limitWithTies_ >= 0), "Top-N must have limit",
-            debugString());
-    Preconditions.checkState(!includeTies_ || (!hasLimit() && limitWithTies_ >= 0),
-            "Top-N with tie handling must set limitWithTies_ only");
+    if (isTypeTopN()) {
+      Preconditions.checkState(hasLimit() ||
+              (includeTies_ && limitWithTies_ >= 0), "Top-N must have limit",
+              debugString());
+      Preconditions.checkState(!includeTies_ || (!hasLimit() && limitWithTies_ >= 0),
+              "Top-N with tie handling must set limitWithTies_ only");
+    }
     Preconditions.checkState(offset_ >= 0);
     msg.node_type = TPlanNodeType.SORT_NODE;
     TSortInfo sort_info = new TSortInfo(Expr.treesToThrift(info_.getSortExprs()),
@@ -295,6 +358,21 @@ public class SortNode extends PlanNode {
     TSortNode sort_node = new TSortNode(sort_info, type_);
     sort_node.setOffset(offset_);
     sort_node.setEstimated_full_input_size(estimatedFullInputSize_);
+
+    if (type_ == TSortType.PARTITIONED_TOPN) {
+      sort_node.setPer_partition_limit(perPartitionLimit_);
+      List<Expr> partExprs = info_.getSortExprs().subList(0, numPartitionExprs_);
+      sort_node.setPartition_exprs(Expr.treesToThrift(partExprs));
+      // Remove the partition exprs for the intra-partition sort.
+      int totalExprs = info_.getSortExprs().size();
+      List<Expr> sortExprs = info_.getSortExprs().subList(numPartitionExprs_, totalExprs);
+      sort_node.setIntra_partition_sort_info(new TSortInfo(Expr.treesToThrift(sortExprs),
+          info_.getIsAscOrder().subList(numPartitionExprs_, totalExprs),
+          info_.getNullsFirst().subList(numPartitionExprs_, totalExprs),
+          info_.getSortingOrder()));
+    }
+    Preconditions.checkState(type_ == TSortType.PARTITIONED_TOPN ||
+            type_ == TSortType.TOPN || !includeTies_);
     sort_node.setInclude_ties(includeTies_);
     if (includeTies_) {
       sort_node.setLimit_with_ties(limitWithTies_);
@@ -309,12 +387,36 @@ public class SortNode extends PlanNode {
     output.append(String.format("%s%s:%s%s\n", prefix, id_.toString(),
         displayName_, getNodeExplainDetail(detailLevel)));
     if (detailLevel.ordinal() >= TExplainLevel.STANDARD.ordinal()) {
-      output.append(detailPrefix + "order by: ");
-      output.append(getSortingOrderExplainString(info_.getSortExprs(),
-          info_.getIsAscOrder(), info_.getNullsFirstParams(), info_.getSortingOrder(),
-          info_.getNumLexicalKeysInZOrder()));
-      if (includeTies_) {
-        output.append(detailPrefix + "limit with ties: " + limitWithTies_ + "\n");
+      if (type_ == TSortType.PARTITIONED_TOPN) {
+        output.append(detailPrefix + "partition by:");
+        List<Expr> partExprs = info_.getSortExprs().subList(0, numPartitionExprs_);
+        if (partExprs.size() > 0) {
+          output.append(" ");
+          output.append(Expr.toSql(partExprs, ToSqlOptions.DEFAULT));
+        }
+        int totalExprs = info_.getSortExprs().size();
+        List<Expr> sortExprs =
+              info_.getSortExprs().subList(numPartitionExprs_, totalExprs);
+        output.append("\n" + detailPrefix + "order by: ");
+        output.append(getSortingOrderExplainString(sortExprs,
+              info_.getIsAscOrder().subList(numPartitionExprs_, totalExprs),
+              info_.getNullsFirstParams().subList(numPartitionExprs_, totalExprs),
+              info_.getSortingOrder(), info_.getNumLexicalKeysInZOrder()));
+        output.append(detailPrefix + "partition limit: " + perPartitionLimit_);
+        if (includeTies_) output.append(" (include ties)");
+        output.append("\n");
+      } else {
+        output.append(detailPrefix + "order by: ");
+        output.append(getSortingOrderExplainString(info_.getSortExprs(),
+            info_.getIsAscOrder(), info_.getNullsFirstParams(), info_.getSortingOrder(),
+            info_.getNumLexicalKeysInZOrder()));
+        if (includeTies_) {
+          output.append(detailPrefix + "limit with ties: " + limitWithTies_ + "\n");
+        }
+      }
+      if (limitSrcPred_ != null) {
+        output.append(detailPrefix + "source expr: " +
+                limitSrcPred_.toSql(ToSqlOptions.SHOW_IMPLICIT_CASTS) + "\n");
       }
     }
 
@@ -360,11 +462,6 @@ public class SortNode extends PlanNode {
       return;
     }
 
-    // For an external sort, set the memory cost to be what is required for a 2-phase
-    // sort. If the input to be sorted would take up N blocks in memory, then the
-    // memory required for a 2-phase sort is sqrt(N) blocks. A single run would be of
-    // size sqrt(N) blocks, and we could merge sqrt(N) such runs with sqrt(N) blocks
-    // of memory.
     double fullInputSize = getChild(0).cardinality_ * avgRowSize_;
     estimatedFullInputSize_ = fullInputSize < 0 ? -1 : (long) Math.ceil(fullInputSize);
     boolean usesVarLenBlocks = false;
@@ -396,10 +493,24 @@ public class SortNode extends PlanNode {
           Math.min((long) Math.ceil(fullInputSize), mem_limit);
       perInstanceMinMemReservation = bufferSize * pageMultiplier;
     } else {
+      Preconditions.checkState(type_ == TSortType.TOTAL ||
+          type_ == TSortType.PARTITIONED_TOPN);
+      // For an external sort, set the memory cost to be what is required for a 2-phase
+      // sort. If the input to be sorted would take up N blocks in memory, then the
+      // memory required for a 2-phase sort is sqrt(N) blocks. A single run would be of
+      // size sqrt(N) blocks, and we could merge sqrt(N) such runs with sqrt(N) blocks
+      // of memory.
       double numInputBlocks = Math.ceil(fullInputSize / (bufferSize * pageMultiplier));
       perInstanceMemEstimate =
           bufferSize * (long) Math.ceil(Math.sqrt(numInputBlocks));
       perInstanceMinMemReservation = 3 * bufferSize * pageMultiplier;
+
+      if (type_ == TSortType.PARTITIONED_TOPN) {
+        // We may be able to estimate a lower memory requirement based on the size
+        // of in-memory heaps.
+        long totalHeapBytes = getSortInfo().estimateMaterializedSize(cardinality_);
+        perInstanceMemEstimate = Math.min(perInstanceMemEstimate, totalHeapBytes);
+      }
     }
     nodeResourceProfile_ = new ResourceProfileBuilder()
         .setMemEstimateBytes(perInstanceMemEstimate)
@@ -408,7 +519,8 @@ public class SortNode extends PlanNode {
   }
 
   private static String getDisplayName(TSortType type) {
-    if (type == TSortType.TOPN) {
+    if (type == TSortType.TOPN || type == TSortType.PARTITIONED_TOPN) {
+      // The two top-n variants can be distinguished by presence of partitioning exprs.
       return "TOP-N";
     } else if (type == TSortType.PARTIAL) {
       return "PARTIAL SORT";
diff --git a/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java b/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java
index 178c448..2bbf7da 100644
--- a/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/CardinalityTest.java
@@ -735,6 +735,46 @@ public class CardinalityTest extends PlannerTestBase {
   }
 
   @Test
+  public void testPartitionedTopNNode() {
+    // Create the path to the SortNode and SelectNode of interest in a distributed plan.
+    List<Integer> selectPath = Arrays.asList(0);
+    List<Integer> sortPath = Arrays.asList(0, 0, 0);
+
+    // NDV(smallint_col) = 97, NDV(bool_col) = 2
+    // 5 rows per top-N partition
+    // 97 * 2 * 5 = 970
+    String lessThanQuery = "select * from (" +
+            "  select *, row_number() over " +
+            "  (partition by smallint_col, bool_col order by id) as rn" +
+            "  from functional.alltypesagg where id % 777 = 0 or id % 10 = 7) v" +
+            " where rn <= 5";
+    final int EXPECTED_CARDINALITY = 970;
+    // Both TOP-N node and the upper Select  node are expected to return the same number
+    // of nodes - the predicate in the Select is effectively pushed to the top-n, so
+    // its selectivity should be 1.
+    verifyApproxCardinality(lessThanQuery, EXPECTED_CARDINALITY, true, ImmutableSet.of(),
+            sortPath, SortNode.class);
+    verifyApproxCardinality(lessThanQuery, EXPECTED_CARDINALITY, true, ImmutableSet.of(),
+            selectPath, SelectNode.class);
+
+
+    // Any equality predicate results in the same cardinality estimate for the top-n.
+    // It also results in the same estimate for the Select node, but for a different
+    // reason: the NDV of row_number() is estimated as 1, so the equality predicate
+    // has a selectivity estimate of 1.0.
+    String eqQuery = "select * from (" +
+            "  select *, row_number() over " +
+            "  (partition by smallint_col, bool_col order by id) as rn" +
+            "  from functional.alltypesagg where id % 777 = 0 or id % 10 = 7) v" +
+            " where rn = 5";
+    verifyApproxCardinality(eqQuery, EXPECTED_CARDINALITY, true, ImmutableSet.of(),
+            sortPath, SortNode.class);
+    verifyApproxCardinality(eqQuery, EXPECTED_CARDINALITY, true, ImmutableSet.of(),
+            selectPath, SelectNode.class);
+  }
+
+
+  @Test
   public void testSubPlanNode() {
     // Create the path to the SubplanNode of interest
     // in a distributed plan.
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index 3f42933..2849c43 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -202,6 +202,12 @@ public class PlannerTest extends PlannerTestBase {
   }
 
   @Test
+  public void testAnalyticRankPushdown() {
+    runPlannerTestFile("analytic-rank-pushdown");
+  }
+
+
+  @Test
   public void testHbase() {
     runPlannerTestFile("hbase");
   }
@@ -778,7 +784,11 @@ public class PlannerTest extends PlannerTestBase {
   @Test
   public void testSortExprMaterialization() {
     addTestFunction("TestFn", Lists.newArrayList(Type.DOUBLE), false);
-    runPlannerTestFile("sort-expr-materialization",
+    // Avoid conversion of RANK()/ROW_NUMBER() predicates to Top-N limits, which
+    // would interfere with the purpose of this test.
+    TQueryOptions options = defaultQueryOptions();
+    options.setAnalytic_rank_pushdown_threshold(0);
+    runPlannerTestFile("sort-expr-materialization", options,
         ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN));
   }
 
@@ -1083,11 +1093,24 @@ public class PlannerTest extends PlannerTestBase {
   }
 
   /**
-   * Test limit pushdown into analytic sort under applicable conditions
+   * Test limit pushdown into analytic sort in isolation.
    */
   @Test
   public void testLimitPushdownAnalytic() {
-    runPlannerTestFile("limit-pushdown-analytic");
+    // The partitioned top-n optimization interacts with limit pushdown. We run the
+    // basic limit pushdown tests with it disabled.
+    TQueryOptions options = defaultQueryOptions();
+    options.setAnalytic_rank_pushdown_threshold(0);
+    runPlannerTestFile("limit-pushdown-analytic", options);
+  }
+
+  /**
+   * Test limit pushdown into analytic sort with the partitioned top-n transformation
+   * also enabled.
+   */
+  @Test
+  public void testLimitPushdownPartitionedTopN() {
+    runPlannerTestFile("limit-pushdown-partitioned-top-n");
   }
 
   /**
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
index aaa9606..5253b09 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
@@ -1554,7 +1554,7 @@ where
 PLAN-ROOT SINK
 |
 07:SELECT
-|  predicates: min(int_col) < 1, max(int_col) < 2, bigint_col > 10, count(int_col) < 3, sum(int_col) < 4, avg(int_col) < 5, min(int_col) != count(int_col), min(int_col) != avg(int_col), max(int_col) != count(int_col), count(int_col) < bigint_col + 3, sum(int_col) < bigint_col + 4, min(int_col) < bigint_col + 1, max(int_col) < bigint_col + 2, avg(int_col) < bigint_col + 5
+|  predicates: avg(int_col) < 5, count(int_col) < 3, max(int_col) < 2, min(int_col) < 1, sum(int_col) < 4, bigint_col > 10, max(int_col) != count(int_col), min(int_col) != avg(int_col), min(int_col) != count(int_col), count(int_col) < bigint_col + 3, sum(int_col) < bigint_col + 4, avg(int_col) < bigint_col + 5, max(int_col) < bigint_col + 2, min(int_col) < bigint_col + 1
 |  row-size=49B cardinality=73
 |
 06:ANALYTIC
@@ -1596,7 +1596,7 @@ PLAN-ROOT SINK
 PLAN-ROOT SINK
 |
 07:SELECT
-|  predicates: min(int_col) < 1, max(int_col) < 2, bigint_col > 10, count(int_col) < 3, sum(int_col) < 4, avg(int_col) < 5, min(int_col) != count(int_col), min(int_col) != avg(int_col), max(int_col) != count(int_col), count(int_col) < bigint_col + 3, sum(int_col) < bigint_col + 4, min(int_col) < bigint_col + 1, max(int_col) < bigint_col + 2, avg(int_col) < bigint_col + 5
+|  predicates: avg(int_col) < 5, count(int_col) < 3, max(int_col) < 2, min(int_col) < 1, sum(int_col) < 4, bigint_col > 10, max(int_col) != count(int_col), min(int_col) != avg(int_col), min(int_col) != count(int_col), count(int_col) < bigint_col + 3, sum(int_col) < bigint_col + 4, avg(int_col) < bigint_col + 5, max(int_col) < bigint_col + 2, min(int_col) < bigint_col + 1
 |  row-size=49B cardinality=73
 |
 06:ANALYTIC
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-rank-pushdown.test b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-rank-pushdown.test
new file mode 100644
index 0000000..8ec8bd3
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-rank-pushdown.test
@@ -0,0 +1,1678 @@
+# Predicate on row_number() without partition by is converted into top-n
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk < 100 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+01:TOP-N [LIMIT=99]
+|  order by: string_col DESC
+|  source expr: row_number() < CAST(100 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: string_col DESC
+|  limit: 99
+|
+01:TOP-N [LIMIT=99]
+|  order by: string_col DESC
+|  source expr: row_number() < CAST(100 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Predicate on row_number() just under the default top-n conversion threshold of 1000.
+select *
+from (
+  select *,
+      row_number() over (order by l_quantity) rnk
+  from tpch.lineitem) v
+where rnk <= 1000
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() <= 1000
+|  row-size=239B cardinality=1.00K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=239B cardinality=1.00K
+|
+01:TOP-N [LIMIT=1000]
+|  order by: l_quantity ASC
+|  source expr: row_number() <= CAST(1000 AS BIGINT)
+|  row-size=231B cardinality=1.00K
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=231B cardinality=6.00M
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() <= 1000
+|  row-size=239B cardinality=1.00K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=239B cardinality=1.00K
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: l_quantity ASC
+|  limit: 1000
+|
+01:TOP-N [LIMIT=1000]
+|  order by: l_quantity ASC
+|  source expr: row_number() <= CAST(1000 AS BIGINT)
+|  row-size=231B cardinality=1.00K
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=231B cardinality=6.00M
+====
+# Predicate on row_number() without partition by is converted into full sort
+# because it exceeds the default top-n conversion threshold of 1000.
+select *, rnk
+from (
+  select *,
+      row_number() over (order by l_quantity) rnk
+  from tpch.lineitem) v
+where rnk <= 1001
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() <= 1001
+|  row-size=239B cardinality=600.12K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=239B cardinality=6.00M
+|
+01:SORT
+|  order by: l_quantity ASC
+|  row-size=231B cardinality=6.00M
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=231B cardinality=6.00M
+====
+# Predicate on row_number() without partition by is converted into full sort
+# because it exceeds the top-n bytes limit (it is under the rank pushdown threshold).
+select *, rnk
+from (
+  select *,
+      row_number() over (order by l_quantity) rnk
+  from tpch.lineitem) v
+where rnk < 999999999
+---- QUERYOPTIONS
+ANALYTIC_RANK_PUSHDOWN_THRESHOLD=9999999999
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < 999999999
+|  row-size=239B cardinality=6.00M
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=239B cardinality=6.00M
+|
+01:SORT [LIMIT=999999998]
+|  order by: l_quantity ASC
+|  source expr: row_number() < CAST(999999999 AS BIGINT)
+|  row-size=231B cardinality=6.00M
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=231B cardinality=6.00M
+====
+# Predicate on row_number() without partition by is converted into top-n
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk < 100 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+01:TOP-N [LIMIT=99]
+|  order by: string_col DESC
+|  source expr: row_number() < CAST(100 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: string_col DESC
+|  limit: 99
+|
+01:TOP-N [LIMIT=99]
+|  order by: string_col DESC
+|  source expr: row_number() < CAST(100 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Predicate on rank() without partition by is converted into unpartitioned
+# top-n with tie handling enabled.
+select * from (
+  select id, int_col,
+      rank() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk < 100 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, rank() < 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: rank()
+|  order by: string_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+01:TOP-N
+|  order by: string_col DESC
+|  limit with ties: 99
+|  source expr: rank() < CAST(100 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, rank() < 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: rank()
+|  order by: string_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+05:TOP-N
+|  order by: string_col DESC
+|  limit with ties: 99
+|  row-size=23B cardinality=99
+|
+04:EXCHANGE [UNPARTITIONED]
+|
+01:TOP-N
+|  order by: string_col DESC
+|  limit with ties: 99
+|  source expr: rank() < CAST(100 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Predicate on row_number() with partition by is converted into partitioned top-n
+select * from (
+  select id, int_col,
+      row_number() over (partition by int_col order by string_col) as rnk
+  from functional.alltypesagg) v
+where rnk < 5 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < 5
+|  row-size=31B cardinality=3.83K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=3.83K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=23B cardinality=3.83K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   predicates: functional.alltypesagg.int_col IS NOT NULL
+   row-size=23B cardinality=10.98K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+06:EXCHANGE [UNPARTITIONED]
+|
+03:SELECT
+|  predicates: row_number() < 5
+|  row-size=31B cardinality=3.83K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=3.83K
+|
+05:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  row-size=23B cardinality=3.83K
+|
+04:EXCHANGE [HASH(int_col)]
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=23B cardinality=3.83K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   predicates: functional.alltypesagg.int_col IS NOT NULL
+   row-size=23B cardinality=10.98K
+====
+# Predicate on rank() with partition by is converted into partitioned top-n
+select * from (
+  select id, int_col,
+      row_number() over (partition by int_col order by string_col) as rnk
+  from functional.alltypesagg) v
+where rnk < 5 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < 5
+|  row-size=31B cardinality=3.83K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=3.83K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=23B cardinality=3.83K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   predicates: functional.alltypesagg.int_col IS NOT NULL
+   row-size=23B cardinality=10.98K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+06:EXCHANGE [UNPARTITIONED]
+|
+03:SELECT
+|  predicates: row_number() < 5
+|  row-size=31B cardinality=3.83K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=3.83K
+|
+05:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  row-size=23B cardinality=3.83K
+|
+04:EXCHANGE [HASH(int_col)]
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=23B cardinality=3.83K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   predicates: functional.alltypesagg.int_col IS NOT NULL
+   row-size=23B cardinality=10.98K
+====
+# Predicate on row_number() with partition by is converted into full sort
+# because it exceeds the default top-n conversion threshold of 1000.
+select *, rnk
+from (
+  select *,
+      row_number() over (partition by l_orderkey order by l_quantity) rnk
+  from tpch.lineitem) v
+where rnk <= 1001
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() <= 1001
+|  row-size=239B cardinality=600.12K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: l_orderkey
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=239B cardinality=6.00M
+|
+01:SORT
+|  order by: l_orderkey ASC NULLS LAST, l_quantity ASC
+|  row-size=231B cardinality=6.00M
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=231B cardinality=6.00M
+====
+# Predicate on row_number() with partition by does not get converted into full sort
+# - the top-n bytes limit does not apply since the partitioned top-n can spill.
+select *, rnk
+from (
+  select *,
+      row_number() over (partition by l_orderkey order by l_quantity) rnk
+  from tpch.lineitem) v
+where rnk < 999999999
+---- QUERYOPTIONS
+ANALYTIC_RANK_PUSHDOWN_THRESHOLD=9999999999
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < 999999999
+|  row-size=239B cardinality=6.00M
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: l_orderkey
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=239B cardinality=6.00M
+|
+01:TOP-N
+|  partition by: l_orderkey
+|  order by: l_quantity ASC
+|  partition limit: 999999998
+|  source expr: row_number() < CAST(999999999 AS BIGINT)
+|  row-size=231B cardinality=6.00M
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=231B cardinality=6.00M
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+06:EXCHANGE [UNPARTITIONED]
+|
+03:SELECT
+|  predicates: row_number() < 999999999
+|  row-size=239B cardinality=6.00M
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: l_orderkey
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=239B cardinality=6.00M
+|
+05:TOP-N
+|  partition by: l_orderkey
+|  order by: l_quantity ASC
+|  partition limit: 999999998
+|  row-size=231B cardinality=6.00M
+|
+04:EXCHANGE [HASH(l_orderkey)]
+|
+01:TOP-N
+|  partition by: l_orderkey
+|  order by: l_quantity ASC
+|  partition limit: 999999998
+|  source expr: row_number() < CAST(999999999 AS BIGINT)
+|  row-size=231B cardinality=6.00M
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=231B cardinality=6.00M
+====
+# Predicates are normalized, so flipped predicates can be pushed down.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where 99 >= rnk and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() <= 99
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+01:TOP-N [LIMIT=99]
+|  order by: string_col DESC
+|  source expr: row_number() <= CAST(99 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() <= 99
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: string_col DESC
+|  limit: 99
+|
+01:TOP-N [LIMIT=99]
+|  order by: string_col DESC
+|  source expr: row_number() <= CAST(99 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Partition by multiple columns.
+select * from (
+  select id, int_col,
+      row_number() over (partition by year, month order by bigint_col) as rnk
+  from functional.alltypesagg) v
+where rnk < 5 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 5
+|  row-size=32B cardinality=1
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: `year`, `month`
+|  order by: bigint_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=32B cardinality=4
+|
+01:TOP-N
+|  partition by: year, month
+|  order by: bigint_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=24B cardinality=4
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=24B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+06:EXCHANGE [UNPARTITIONED]
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 5
+|  row-size=32B cardinality=1
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: `year`, `month`
+|  order by: bigint_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=32B cardinality=4
+|
+05:TOP-N
+|  partition by: year, month
+|  order by: bigint_col ASC
+|  partition limit: 4
+|  row-size=24B cardinality=4
+|
+04:EXCHANGE [HASH(year,month)]
+|
+01:TOP-N
+|  partition by: year, month
+|  order by: bigint_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=24B cardinality=4
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=24B cardinality=11.00K
+====
+# dense_rank() is not eligible for push down.
+select * from (
+  select id, int_col,
+      dense_rank() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk < 100 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, dense_rank() < 100
+|  row-size=31B cardinality=1.10K
+|
+02:ANALYTIC
+|  functions: dense_rank()
+|  order by: string_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=11.00K
+|
+01:SORT
+|  order by: string_col DESC
+|  row-size=23B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Analytic predicate can be migrated into inline view (the predicate on rnk),
+# but not through the analytic if there are multiple levels of views
+# (the predicate on int_col).
+select * from (
+  select id, int_col,
+      row_number() over (partition by int_col order by string_col nulls last) as rnk
+  from (
+    select id, string_col, first_value(int_col) over (partition by id) as int_col
+    from functional.alltypes) v2) v
+where rnk < 5 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+05:SELECT
+|  predicates: first_value(int_col) IS NOT NULL, row_number() < 5
+|  row-size=33B cardinality=4
+|
+04:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=33B cardinality=40
+|
+03:TOP-N
+|  partition by: first_value(int_col)
+|  order by: string_col ASC NULLS LAST
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=25B cardinality=40
+|
+02:ANALYTIC
+|  functions: first_value(int_col)
+|  partition by: id
+|  row-size=25B cardinality=7.30K
+|
+01:SORT
+|  order by: id ASC NULLS LAST
+|  row-size=21B cardinality=7.30K
+|
+00:SCAN HDFS [functional.alltypes]
+   HDFS partitions=24/24 files=24 size=478.45KB
+   row-size=21B cardinality=7.30K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+09:EXCHANGE [UNPARTITIONED]
+|
+05:SELECT
+|  predicates: first_value(int_col) IS NOT NULL, row_number() < 5
+|  row-size=33B cardinality=4
+|
+04:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=33B cardinality=40
+|
+08:TOP-N
+|  partition by: first_value(int_col)
+|  order by: string_col ASC NULLS LAST
+|  partition limit: 4
+|  row-size=25B cardinality=40
+|
+07:EXCHANGE [HASH(first_value(int_col))]
+|
+03:TOP-N
+|  partition by: first_value(int_col)
+|  order by: string_col ASC NULLS LAST
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=25B cardinality=40
+|
+02:ANALYTIC
+|  functions: first_value(int_col)
+|  partition by: id
+|  row-size=25B cardinality=7.30K
+|
+01:SORT
+|  order by: id ASC NULLS LAST
+|  row-size=21B cardinality=7.30K
+|
+06:EXCHANGE [HASH(id)]
+|
+00:SCAN HDFS [functional.alltypes]
+   HDFS partitions=24/24 files=24 size=478.45KB
+   row-size=21B cardinality=7.30K
+====
+# Analytic predicate cannot be pushed down if subquery has a limit.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg
+  limit 1000000) v
+where rnk < 100 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 100
+|  row-size=31B cardinality=1.10K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  limit: 1000000
+|  row-size=31B cardinality=11.00K
+|
+01:TOP-N [LIMIT=1000000]
+|  order by: string_col DESC
+|  row-size=23B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Analytic predicate cannot be pushed into UNION.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rownum
+  from functional.alltypesagg UNION ALL select 1, 2, 3) v
+where rownum <= 20
+---- PLAN
+PLAN-ROOT SINK
+|
+04:SELECT
+|  predicates: rownum <= 20
+|  row-size=16B cardinality=1.10K
+|
+00:UNION
+|  constant-operands=1
+|  row-size=16B cardinality=11.00K
+|
+03:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=11.00K
+|
+02:SORT
+|  order by: string_col DESC
+|  row-size=23B cardinality=11.00K
+|
+01:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# <= predicate
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk <= 100 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() <= 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=100
+|
+01:TOP-N [LIMIT=100]
+|  order by: string_col DESC
+|  source expr: row_number() <= CAST(100 AS BIGINT)
+|  row-size=23B cardinality=100
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() <= 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=100
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: string_col DESC
+|  limit: 100
+|
+01:TOP-N [LIMIT=100]
+|  order by: string_col DESC
+|  source expr: row_number() <= CAST(100 AS BIGINT)
+|  row-size=23B cardinality=100
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# = predicate
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk = 42 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() = 42
+|  row-size=31B cardinality=4
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=42
+|
+01:TOP-N [LIMIT=42]
+|  order by: string_col DESC
+|  source expr: row_number() = CAST(42 AS BIGINT)
+|  row-size=23B cardinality=42
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() = 42
+|  row-size=31B cardinality=4
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=42
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: string_col DESC
+|  limit: 42
+|
+01:TOP-N [LIMIT=42]
+|  order by: string_col DESC
+|  source expr: row_number() = CAST(42 AS BIGINT)
+|  row-size=23B cardinality=42
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# BETWEEN predicate - only the <= is pushed down.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk between 46 and 52
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() >= 46, row_number() <= 52
+|  row-size=31B cardinality=5
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=52
+|
+01:TOP-N [LIMIT=52]
+|  order by: string_col DESC
+|  source expr: row_number() <= CAST(52 AS BIGINT)
+|  row-size=23B cardinality=52
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() >= 46, row_number() <= 52
+|  row-size=31B cardinality=5
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=52
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: string_col DESC
+|  limit: 52
+|
+01:TOP-N [LIMIT=52]
+|  order by: string_col DESC
+|  source expr: row_number() <= CAST(52 AS BIGINT)
+|  row-size=23B cardinality=52
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# > and >= are not pushed down.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk >= 2 and rnk > 3 and rnk < 100
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() > 3, row_number() >= 2, row_number() < 100
+|  row-size=31B cardinality=10
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=99
+|
+01:TOP-N [LIMIT=99]
+|  order by: string_col DESC
+|  source expr: row_number() < CAST(100 AS BIGINT)
+|  row-size=23B cardinality=99
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# The smallest limit is chosen for pushdown.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rownum,
+      row_number() over (order by string_col desc) as rownum2
+  from functional.alltypesagg) v
+where rownum <= 20 and rownum2 < 20 and rownum = 50
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() <= 20, row_number() < 20, row_number() = 50
+|  row-size=31B cardinality=2
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=19
+|
+01:TOP-N [LIMIT=19]
+|  order by: string_col DESC
+|  source expr: row_number() < CAST(20 AS BIGINT)
+|  row-size=23B cardinality=19
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() <= 20, row_number() < 20, row_number() = 50
+|  row-size=31B cardinality=2
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=19
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: string_col DESC
+|  limit: 19
+|
+01:TOP-N [LIMIT=19]
+|  order by: string_col DESC
+|  source expr: row_number() < CAST(20 AS BIGINT)
+|  row-size=23B cardinality=19
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Pushdown does not work with rank() and row_number() combined because of
+# incompatible windows.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rownum,
+      rank() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk < 21 and rownum <= 20 and rnk < 100
+---- PLAN
+PLAN-ROOT SINK
+|
+04:SELECT
+|  predicates: rank() < 100, rank() < 21, row_number() <= 20
+|  row-size=39B cardinality=1.10K
+|
+03:ANALYTIC
+|  functions: rank()
+|  order by: string_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=39B cardinality=11.00K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=11.00K
+|
+01:SORT
+|  order by: string_col DESC
+|  row-size=23B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Limit pushdown is compatible with other analytics with same window.
+select * from (
+  select id, int_col,
+      row_number() over (partition by year order by string_col) as rnk,
+      first_value(tinyint_col) over (partition by year order by string_col)
+  from functional.alltypesagg) v
+where rnk < 5 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 5
+|  row-size=37B cardinality=1
+|
+02:ANALYTIC
+|  functions: row_number(), first_value(tinyint_col)
+|  partition by: `year`
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=37B cardinality=4
+|
+01:TOP-N
+|  partition by: year
+|  order by: string_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=28B cardinality=4
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=28B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+06:EXCHANGE [UNPARTITIONED]
+|
+03:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 5
+|  row-size=37B cardinality=1
+|
+02:ANALYTIC
+|  functions: row_number(), first_value(tinyint_col)
+|  partition by: `year`
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=37B cardinality=4
+|
+05:TOP-N
+|  partition by: year
+|  order by: string_col ASC
+|  partition limit: 4
+|  row-size=28B cardinality=4
+|
+04:EXCHANGE [HASH(year)]
+|
+01:TOP-N
+|  partition by: year
+|  order by: string_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=28B cardinality=4
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=28B cardinality=11.00K
+====
+# Limit cannot be pushed through another analytic in the same select block with
+# a different partitioning.
+select * from (
+  select id, int_col,
+      row_number() over (partition by int_col order by string_col) as rnk,
+      sum(tinyint_col) over (partition by year order by string_col)
+  from functional.alltypesagg) v
+where rnk < 5 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+05:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 5
+|  row-size=44B cardinality=1.10K
+|
+04:ANALYTIC
+|  functions: sum(tinyint_col)
+|  partition by: `year`
+|  order by: string_col ASC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=44B cardinality=11.00K
+|
+03:SORT
+|  order by: year ASC NULLS LAST, string_col ASC
+|  row-size=36B cardinality=11.00K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=36B cardinality=11.00K
+|
+01:SORT
+|  order by: int_col ASC NULLS LAST, string_col ASC
+|  row-size=28B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=28B cardinality=11.00K
+====
+# The limit can be pushed down to the sort if it is the last sort group and
+# there are no incompatible windows in that sort group.
+select * from (
+  select id, int_col,
+      row_number() over (partition by int_col order by string_col) as rnk,
+      first_value(tinyint_col) over (partition by year order by string_col
+                                     rows between 2 preceding and current row)
+  from functional.alltypesagg) v
+where rnk < 5 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+05:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 5
+|  row-size=37B cardinality=383
+|
+04:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=37B cardinality=3.83K
+|
+03:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=29B cardinality=3.83K
+|
+02:ANALYTIC
+|  functions: first_value_rewrite(tinyint_col, -1)
+|  partition by: `year`
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND 2 PRECEDING
+|  row-size=29B cardinality=11.00K
+|
+01:SORT
+|  order by: year ASC NULLS LAST, string_col ASC
+|  row-size=28B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=28B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+09:EXCHANGE [UNPARTITIONED]
+|
+05:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 5
+|  row-size=37B cardinality=383
+|
+04:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=37B cardinality=3.83K
+|
+08:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  row-size=29B cardinality=3.83K
+|
+07:EXCHANGE [HASH(int_col)]
+|
+03:TOP-N
+|  partition by: int_col
+|  order by: string_col ASC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=29B cardinality=3.83K
+|
+02:ANALYTIC
+|  functions: first_value_rewrite(tinyint_col, -1)
+|  partition by: `year`
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND 2 PRECEDING
+|  row-size=29B cardinality=11.00K
+|
+01:SORT
+|  order by: year ASC NULLS LAST, string_col ASC
+|  row-size=28B cardinality=11.00K
+|
+06:EXCHANGE [HASH(`year`)]
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=28B cardinality=11.00K
+====
+# The limit cannot be pushed down to the sort if there are incompatible windows in that
+# sort group.
+select * from (
+  select id, int_col,
+      row_number() over (partition by year order by string_col) as rnk,
+      last_value(tinyint_col) over (partition by year order by string_col
+                                     rows between 2 preceding and current row)
+  from functional.alltypesagg) v
+where rnk < 5 and int_col is not null
+---- PLAN
+PLAN-ROOT SINK
+|
+04:SELECT
+|  predicates: int_col IS NOT NULL, row_number() < 5
+|  row-size=37B cardinality=1.10K
+|
+03:ANALYTIC
+|  functions: row_number()
+|  partition by: `year`
+|  order by: string_col ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=37B cardinality=11.00K
+|
+02:ANALYTIC
+|  functions: last_value(tinyint_col)
+|  partition by: `year`
+|  order by: string_col ASC
+|  window: ROWS BETWEEN 2 PRECEDING AND CURRENT ROW
+|  row-size=29B cardinality=11.00K
+|
+01:SORT
+|  order by: year ASC NULLS LAST, string_col ASC
+|  row-size=28B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=28B cardinality=11.00K
+====
+# Effective limit of 0 results in a limit of 1 in the TOP-N so that it is executable,
+# and the original predicate must be preserved.
+select distinct l_shipmode, l_quantity, rnk
+from (
+  select *,
+    row_number() over (partition by l_shipmode order by l_quantity) rnk,
+    first_value(l_comment) over (partition by l_shipmode order by l_linenumber)
+    from tpch.lineitem) v
+where rnk < 1
+---- PLAN
+PLAN-ROOT SINK
+|
+04:AGGREGATE [FINALIZE]
+|  group by: l_shipmode, l_quantity, row_number()
+|  row-size=32B cardinality=7
+|
+03:SELECT
+|  predicates: row_number() < 1
+|  row-size=32B cardinality=7
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=32B cardinality=7
+|
+01:TOP-N
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  partition limit: 1
+|  source expr: row_number() < CAST(1 AS BIGINT)
+|  row-size=24B cardinality=7
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=24B cardinality=6.00M
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+09:EXCHANGE [UNPARTITIONED]
+|
+08:AGGREGATE [FINALIZE]
+|  group by: l_shipmode, l_quantity, rnk
+|  row-size=32B cardinality=7
+|
+07:EXCHANGE [HASH(l_shipmode,l_quantity,rnk)]
+|
+04:AGGREGATE [STREAMING]
+|  group by: l_shipmode, l_quantity, row_number()
+|  row-size=32B cardinality=7
+|
+03:SELECT
+|  predicates: row_number() < 1
+|  row-size=32B cardinality=7
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=32B cardinality=7
+|
+06:TOP-N
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  partition limit: 1
+|  row-size=24B cardinality=7
+|
+05:EXCHANGE [HASH(l_shipmode)]
+|
+01:TOP-N
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  partition limit: 1
+|  source expr: row_number() < CAST(1 AS BIGINT)
+|  row-size=24B cardinality=7
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=24B cardinality=6.00M
+====
+# Effective negative limits are handled correctly.
+select l_shipmode, l_quantity, rnk
+from (
+  select *,
+    row_number() over (partition by l_shipmode order by l_quantity) rnk,
+    first_value(l_comment) over (partition by l_shipmode order by l_linenumber)
+    from tpch.lineitem) v
+where rnk = -42
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() = -42
+|  row-size=32B cardinality=7
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=32B cardinality=7
+|
+01:TOP-N
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  partition limit: 1
+|  source expr: row_number() = CAST(-42 AS BIGINT)
+|  row-size=24B cardinality=7
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=24B cardinality=6.00M
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+06:EXCHANGE [UNPARTITIONED]
+|
+03:SELECT
+|  predicates: row_number() = -42
+|  row-size=32B cardinality=7
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=32B cardinality=7
+|
+05:TOP-N
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  partition limit: 1
+|  row-size=24B cardinality=7
+|
+04:EXCHANGE [HASH(l_shipmode)]
+|
+01:TOP-N
+|  partition by: l_shipmode
+|  order by: l_quantity ASC
+|  partition limit: 1
+|  source expr: row_number() = CAST(-42 AS BIGINT)
+|  row-size=24B cardinality=7
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   row-size=24B cardinality=6.00M
+====
+# Other predicates referencing rank() and row_number() can't be
+# converted into limits.
+select * from (
+  select id, int_col,
+        row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk % 100 = 10 and case when id % 2 = 0 then rnk > 1 else true end
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() % 100 = 10, CASE WHEN id % 2 = 0 THEN row_number() > 1 ELSE TRUE END
+|  row-size=31B cardinality=1.10K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=11.00K
+|
+01:SORT
+|  order by: string_col DESC
+|  row-size=23B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Predicates referencing rank() nested in OR can't be converted into limits.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk < 10 or id = 10
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < 10 OR id = 10
+|  row-size=31B cardinality=1.10K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=11.00K
+|
+01:SORT
+|  order by: string_col DESC
+|  row-size=23B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Comparison with non-constant rank predicate can't be converted into limit.
+select * from (
+  select id, int_col,
+      row_number() over (order by string_col desc) as rnk
+  from functional.alltypesagg) v
+where rnk < int_col
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < int_col
+|  row-size=31B cardinality=1.10K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=11.00K
+|
+01:SORT
+|  order by: string_col DESC
+|  row-size=23B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=23B cardinality=11.00K
+====
+# Partition by literal is converted into unpartitioned top-n
+select * from (
+  select id, int_col,
+      row_number() over (partition by 1 order by string_col desc) as rnk
+  from functional.alltypesagg
+  where int_col = 1) v
+where rnk < 5
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < 5
+|  row-size=31B cardinality=4
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: 1
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=4
+|
+01:TOP-N [LIMIT=4]
+|  order by: 1 ASC NULLS LAST, string_col DESC
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=23B cardinality=4
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   predicates: int_col = 1
+   row-size=23B cardinality=11
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < 5
+|  row-size=31B cardinality=4
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: 1
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=4
+|
+04:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: 1 ASC NULLS LAST, string_col DESC
+|  limit: 4
+|
+01:TOP-N [LIMIT=4]
+|  order by: 1 ASC NULLS LAST, string_col DESC
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=23B cardinality=4
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   predicates: int_col = 1
+   row-size=23B cardinality=11
+====
+# Partition by a non-literal expression that is constant is converted into partitioned
+# top-n, but could actually be handled by unpartitioned top-n correctly.
+select * from (
+  select id, int_col,
+      row_number() over (partition by int_col order by string_col desc) as rnk
+  from functional.alltypesagg
+  where int_col = 1) v
+where rnk < 5
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: row_number() < 5
+|  row-size=31B cardinality=11
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=11
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: string_col DESC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=23B cardinality=11
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   predicates: int_col = 1
+   row-size=23B cardinality=11
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+06:EXCHANGE [UNPARTITIONED]
+|
+03:SELECT
+|  predicates: row_number() < 5
+|  row-size=31B cardinality=11
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: string_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=31B cardinality=11
+|
+05:TOP-N
+|  partition by: int_col
+|  order by: string_col DESC
+|  partition limit: 4
+|  row-size=23B cardinality=11
+|
+04:EXCHANGE [HASH(int_col)]
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: string_col DESC
+|  partition limit: 4
+|  source expr: row_number() < CAST(5 AS BIGINT)
+|  row-size=23B cardinality=11
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   predicates: int_col = 1
+   row-size=23B cardinality=11
+====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/limit-pushdown-partitioned-top-n.test b/testdata/workloads/functional-planner/queries/PlannerTest/limit-pushdown-partitioned-top-n.test
new file mode 100644
index 0000000..1679321
--- /dev/null
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/limit-pushdown-partitioned-top-n.test
@@ -0,0 +1,856 @@
+# IMPALA-9983
+# Base case. Limit pushdown into analytic sort should be applied
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+order by int_col, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=20
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=20
+|
+01:TOP-N
+|  order by: int_col ASC NULLS LAST, smallint_col DESC
+|  limit with ties: 20
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=20
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+07:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  limit: 10
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=20
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=20
+|
+06:TOP-N
+|  order by: int_col ASC NULLS LAST, smallint_col DESC
+|  limit with ties: 20
+|  row-size=14B cardinality=20
+|
+05:EXCHANGE [HASH(int_col)]
+|
+01:TOP-N
+|  order by: int_col ASC NULLS LAST, smallint_col DESC
+|  limit with ties: 20
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=20
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# row_number() predicate on equality instead of range.
+# The rk = 10 predicate allows use of the partitioned top-n for the
+# analytic sort, but the limit from the final TOP-N cannot be pushed
+# down safely, since the SELECT will filter out additional rows per partition.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    row_number() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk = 10
+order by int_col, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: row_number() = 10
+|  row-size=22B cardinality=9.57K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10
+|  source expr: row_number() = CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+07:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  limit: 10
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: row_number() = 10
+|  row-size=22B cardinality=9.57K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+06:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10
+|  row-size=14B cardinality=9.57K
+|
+05:EXCHANGE [HASH(int_col)]
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10
+|  source expr: row_number() = CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Limit pushdown into analytic sort cannot be applied because
+# per-partition limit in partitioned top-n is lower than the limit
+# to be pushed down.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    row_number() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk = 5
+order by int_col, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: row_number() = 5
+|  row-size=22B cardinality=4.79K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=4.79K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 5
+|  source expr: row_number() = CAST(5 AS BIGINT)
+|  row-size=14B cardinality=4.79K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+07:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  limit: 10
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: row_number() = 5
+|  row-size=22B cardinality=4.79K
+|
+02:ANALYTIC
+|  functions: row_number()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=4.79K
+|
+06:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 5
+|  row-size=14B cardinality=4.79K
+|
+05:EXCHANGE [HASH(int_col)]
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 5
+|  source expr: row_number() = CAST(5 AS BIGINT)
+|  row-size=14B cardinality=4.79K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Multi column partition-by which is prefix of sort exprs.
+# Limit pushdown into analytic sort should be applied
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col, bigint_col
+                 order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+order by int_col, bigint_col, smallint_col, rk
+limit 10;
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=20
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col, bigint_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=20
+|
+01:TOP-N
+|  order by: int_col ASC NULLS LAST, bigint_col ASC NULLS LAST, smallint_col DESC
+|  limit with ties: 20
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=20
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+07:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  limit: 10
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=20
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col, bigint_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=20
+|
+06:TOP-N
+|  order by: int_col ASC NULLS LAST, bigint_col ASC NULLS LAST, smallint_col DESC
+|  limit with ties: 20
+|  row-size=14B cardinality=20
+|
+05:EXCHANGE [HASH(int_col,bigint_col)]
+|
+01:TOP-N
+|  order by: int_col ASC NULLS LAST, bigint_col ASC NULLS LAST, smallint_col DESC
+|  limit with ties: 20
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=20
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Limit is present without Order By.
+# Limit pushdown should be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  limit: 10
+|  row-size=22B cardinality=10
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+06:EXCHANGE [UNPARTITIONED]
+|  limit: 10
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  limit: 10
+|  row-size=22B cardinality=10
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+05:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10 (include ties)
+|  row-size=14B cardinality=9.57K
+|
+04:EXCHANGE [HASH(int_col)]
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# rank() predicate operands is flipped.
+# Limit pushdown should be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where 10 > rk
+order by int_col, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() < 10
+|  row-size=22B cardinality=8.61K
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=8.61K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 9 (include ties)
+|  source expr: rank() < CAST(10 AS BIGINT)
+|  row-size=14B cardinality=8.61K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
+07:MERGING-EXCHANGE [UNPARTITIONED]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  limit: 10
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() < 10
+|  row-size=22B cardinality=8.61K
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=8.61K
+|
+06:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 9 (include ties)
+|  row-size=14B cardinality=8.61K
+|
+05:EXCHANGE [HASH(int_col)]
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 9 (include ties)
+|  source expr: rank() < CAST(10 AS BIGINT)
+|  row-size=14B cardinality=8.61K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Partition-by and order-by exprs refer to derived table
+# columns rather than base table columns.
+# Limit pushdown should be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from (select int_col, bigint_col, smallint_col from functional.alltypesagg
+        group by int_col, bigint_col, smallint_col)dt1)dt2
+where rk <= 10
+order by int_col, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+05:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+04:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=20
+|
+03:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=20
+|
+02:TOP-N
+|  order by: int_col ASC NULLS LAST, smallint_col DESC
+|  limit with ties: 20
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=20
+|
+01:AGGREGATE [FINALIZE]
+|  group by: int_col, bigint_col, smallint_col
+|  row-size=14B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# No limit in the query.
+# Limit pushdown should not be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+order by int_col, bigint_col, smallint_col, rk
+---- PLAN
+PLAN-ROOT SINK
+|
+04:SORT
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=9.57K
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=9.57K
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Rank predicate's upper bound is greater than limit.
+# Limit pushdown should not be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 20
+order by int_col, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() <= 20
+|  row-size=22B cardinality=30
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=30
+|
+01:TOP-N
+|  order by: int_col ASC NULLS LAST, smallint_col DESC
+|  limit with ties: 30
+|  source expr: rank() <= CAST(20 AS BIGINT)
+|  row-size=14B cardinality=30
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Function is not a ranking function.
+# Limit pushdown should not be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    min(double_col) over (partition by int_col
+                          order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+order by int_col, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: min(double_col) <= 10
+|  row-size=30B cardinality=1.10K
+|
+02:ANALYTIC
+|  functions: min(double_col)
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=30B cardinality=11.00K
+|
+01:SORT
+|  order by: int_col ASC NULLS LAST, smallint_col DESC
+|  row-size=22B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=22B cardinality=11.00K
+====
+# Partition-by expr is not a prefix of the sort exprs.
+# Limit pushdown should not be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+order by bigint_col, int_col, smallint_col, rk
+limit 10;
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: bigint_col ASC, int_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=9.57K
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Blocking operator occurs between the top level TopN
+# operator and the analytic operator.
+# Limit pushdown should not be applied.
+select int_col, bigint_col, smallint_col, rk from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+group by int_col, bigint_col, smallint_col, rk
+order by int_col, bigint_col, smallint_col, rk
+limit 10;
+---- PLAN
+PLAN-ROOT SINK
+|
+05:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+04:AGGREGATE [FINALIZE]
+|  group by: int_col, bigint_col, smallint_col, rank()
+|  row-size=22B cardinality=9.57K
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=9.57K
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Rank predicate is not one of <, =, <= .
+# Limit pushdown should not be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk in (10, 20, 30)
+order by int_col, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() IN (10, 20, 30)
+|  row-size=22B cardinality=11.00K
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=11.00K
+|
+01:SORT
+|  order by: int_col ASC NULLS LAST, smallint_col DESC
+|  row-size=14B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# 2 ranking functions with different partition-by exprs. Predicate is on
+# the dense_rank() function. If the dense_rank() appears as the top
+# level analytic function, the limit pushdown can be applied. Otherwise,
+# it should not.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    dense_rank() over (partition by bigint_col order by smallint_col desc) rk2,
+    rank() over (partition by int_col order by smallint_col desc) rk1
+  from functional.alltypesagg) dt
+where rk2 <= 10
+order by int_col, bigint_col, smallint_col, rk1, rk2
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+06:TOP-N [LIMIT=10]
+|  order by: int_col ASC, bigint_col ASC, smallint_col ASC, rk1 ASC, rk2 ASC
+|  row-size=30B cardinality=10
+|
+05:SELECT
+|  predicates: dense_rank() <= 10
+|  row-size=30B cardinality=1.10K
+|
+04:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=30B cardinality=11.00K
+|
+03:SORT
+|  order by: int_col ASC NULLS LAST, smallint_col DESC
+|  row-size=22B cardinality=11.00K
+|
+02:ANALYTIC
+|  functions: dense_rank()
+|  partition by: bigint_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=11.00K
+|
+01:SORT
+|  order by: bigint_col ASC NULLS LAST, smallint_col DESC
+|  row-size=14B cardinality=11.00K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# Asc/Desc direction is different for the top order-by.
+# The rk <= 10 predicate allows use of the partitioned top-n for the
+# analytic sort, but the limit from the final TOP-N cannot be pushed
+# down.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+order by int_col DESC, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col DESC, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=9.57K
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
+# NULLS FIRST for the top order-by which is different compared
+# to the partition-by expr.
+# Limit pushdown should not be applied.
+select * from (
+  select int_col, bigint_col, smallint_col,
+    rank() over (partition by int_col order by smallint_col desc) rk
+  from functional.alltypesagg) dt
+where rk <= 10
+order by int_col NULLS FIRST, bigint_col, smallint_col, rk
+limit 10
+---- PLAN
+PLAN-ROOT SINK
+|
+04:TOP-N [LIMIT=10]
+|  order by: int_col ASC NULLS FIRST, bigint_col ASC, smallint_col ASC, rk ASC
+|  row-size=22B cardinality=10
+|
+03:SELECT
+|  predicates: rank() <= 10
+|  row-size=22B cardinality=9.57K
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  row-size=22B cardinality=9.57K
+|
+01:TOP-N
+|  partition by: int_col
+|  order by: smallint_col DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  row-size=14B cardinality=9.57K
+|
+00:SCAN HDFS [functional.alltypesagg]
+   HDFS partitions=11/11 files=11 size=814.73KB
+   row-size=14B cardinality=11.00K
+====
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
index 30159a5..0bb9b08 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -2438,6 +2438,197 @@ Per-Instance Resources: mem-estimate=80.02MB mem-reservation=40.00MB thread-rese
    tuple-ids=0 row-size=231B cardinality=6.00M
    in pipelines: 00(GETNEXT)
 ====
+# Partitioned TOP-N
+# Memory estimate reflects estimated size of in-memory
+# heap per partition.
+select * from (
+  select l_shipdate, l_quantity, l_orderkey,
+    rank() over (partition by l_shipdate order by l_orderkey desc) rk
+  from tpch.lineitem) dt
+where rk <= 10
+---- PLAN
+Max Per-Host Resource Reservation: Memory=24.00MB Threads=2
+Per-Host Resource Estimates: Memory=104MB
+Analyzed query: SELECT * FROM (SELECT l_shipdate, l_quantity, l_orderkey, rank()
+OVER (PARTITION BY l_shipdate ORDER BY l_orderkey DESC) rk FROM tpch.lineitem)
+dt WHERE rk <= CAST(10 AS BIGINT)
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=104.00MB mem-reservation=24.00MB thread-reservation=2
+PLAN-ROOT SINK
+|  output exprs: l_shipdate, l_quantity, l_orderkey, rank()
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|
+03:SELECT
+|  predicates: rank() <= CAST(10 AS BIGINT)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=6,5 row-size=46B cardinality=26.29K
+|  in pipelines: 01(GETNEXT)
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: l_shipdate
+|  order by: l_orderkey DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=6,5 row-size=46B cardinality=26.29K
+|  in pipelines: 01(GETNEXT)
+|
+01:TOP-N
+|  partition by: l_shipdate
+|  order by: l_orderkey DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=6 row-size=38B cardinality=26.29K
+|  in pipelines: 01(GETNEXT), 00(OPEN)
+|
+00:SCAN HDFS [tpch.lineitem]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   stored statistics:
+     table: rows=6.00M size=718.94MB
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=1.07M
+   mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
+   tuple-ids=0 row-size=38B cardinality=6.00M
+   in pipelines: 00(GETNEXT)
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=36.00MB Threads=4
+Per-Host Resource Estimates: Memory=117MB
+Analyzed query: SELECT * FROM (SELECT l_shipdate, l_quantity, l_orderkey, rank()
+OVER (PARTITION BY l_shipdate ORDER BY l_orderkey DESC) rk FROM tpch.lineitem)
+dt WHERE rk <= CAST(10 AS BIGINT)
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=555.67KB mem-reservation=0B thread-reservation=1
+PLAN-ROOT SINK
+|  output exprs: l_shipdate, l_quantity, l_orderkey, rank()
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|
+06:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=555.67KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6,5 row-size=46B cardinality=26.29K
+|  in pipelines: 05(GETNEXT)
+|
+F01:PLAN FRAGMENT [HASH(l_shipdate)] hosts=3 instances=3
+Per-Host Resources: mem-estimate=16.44MB mem-reservation=16.00MB thread-reservation=1
+03:SELECT
+|  predicates: rank() <= CAST(10 AS BIGINT)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=6,5 row-size=46B cardinality=26.29K
+|  in pipelines: 05(GETNEXT)
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: l_shipdate
+|  order by: l_orderkey DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=6,5 row-size=46B cardinality=26.29K
+|  in pipelines: 05(GETNEXT)
+|
+05:TOP-N
+|  partition by: l_shipdate
+|  order by: l_orderkey DESC
+|  partition limit: 10 (include ties)
+|  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=6 row-size=38B cardinality=26.29K
+|  in pipelines: 05(GETNEXT), 01(OPEN)
+|
+04:EXCHANGE [HASH(l_shipdate)]
+|  mem-estimate=451.20KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=38B cardinality=26.29K
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=100.00MB mem-reservation=20.00MB thread-reservation=2
+01:TOP-N
+|  partition by: l_shipdate
+|  order by: l_orderkey DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=6 row-size=38B cardinality=26.29K
+|  in pipelines: 01(GETNEXT), 00(OPEN)
+|
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   stored statistics:
+     table: rows=6.00M size=718.94MB
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=1.07M
+   mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=1
+   tuple-ids=0 row-size=38B cardinality=6.00M
+   in pipelines: 00(GETNEXT)
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=72.00MB Threads=5
+Per-Host Resource Estimates: Memory=234MB
+Analyzed query: SELECT * FROM (SELECT l_shipdate, l_quantity, l_orderkey, rank()
+OVER (PARTITION BY l_shipdate ORDER BY l_orderkey DESC) rk FROM tpch.lineitem)
+dt WHERE rk <= CAST(10 AS BIGINT)
+
+F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Instance Resources: mem-estimate=717.67KB mem-reservation=0B thread-reservation=1
+PLAN-ROOT SINK
+|  output exprs: l_shipdate, l_quantity, l_orderkey, rank()
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|
+06:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=717.67KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6,5 row-size=46B cardinality=26.29K
+|  in pipelines: 05(GETNEXT)
+|
+F01:PLAN FRAGMENT [HASH(l_shipdate)] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=16.56MB mem-reservation=16.00MB thread-reservation=1
+03:SELECT
+|  predicates: rank() <= CAST(10 AS BIGINT)
+|  mem-estimate=0B mem-reservation=0B thread-reservation=0
+|  tuple-ids=6,5 row-size=46B cardinality=26.29K
+|  in pipelines: 05(GETNEXT)
+|
+02:ANALYTIC
+|  functions: rank()
+|  partition by: l_shipdate
+|  order by: l_orderkey DESC
+|  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
+|  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=6,5 row-size=46B cardinality=26.29K
+|  in pipelines: 05(GETNEXT)
+|
+05:TOP-N
+|  partition by: l_shipdate
+|  order by: l_orderkey DESC
+|  partition limit: 10 (include ties)
+|  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=6 row-size=38B cardinality=26.29K
+|  in pipelines: 05(GETNEXT), 01(OPEN)
+|
+04:EXCHANGE [HASH(l_shipdate)]
+|  mem-estimate=577.20KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=6 row-size=38B cardinality=26.29K
+|  in pipelines: 01(GETNEXT)
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Instance Resources: mem-estimate=100.00MB mem-reservation=20.00MB thread-reservation=1
+01:TOP-N
+|  partition by: l_shipdate
+|  order by: l_orderkey DESC
+|  partition limit: 10 (include ties)
+|  source expr: rank() <= CAST(10 AS BIGINT)
+|  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
+|  tuple-ids=6 row-size=38B cardinality=26.29K
+|  in pipelines: 01(GETNEXT), 00(OPEN)
+|
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   HDFS partitions=1/1 files=1 size=718.94MB
+   stored statistics:
+     table: rows=6.00M size=718.94MB
+     columns: all
+   extrapolated-rows=disabled max-scan-range-rows=1.07M
+   mem-estimate=88.00MB mem-reservation=8.00MB thread-reservation=0
+   tuple-ids=0 row-size=38B cardinality=6.00M
+   in pipelines: 00(GETNEXT)
+====
 # Broadcast Hash Join
 select *
 from tpch.lineitem inner join tpch.orders on l_orderkey = o_orderkey
@@ -6137,14 +6328,15 @@ PLAN-ROOT SINK
 01:TOP-N
 |  order by: l_partkey ASC NULLS LAST, l_orderkey DESC
 |  limit with ties: 200
+|  source expr: rank() <= CAST(100 AS BIGINT)
 |  mem-estimate=4.69KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=8 row-size=24B cardinality=200
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
-   HDFS partitions=1/1 files=3 size=193.98MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.98MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=16.00MB thread-reservation=1
@@ -6211,14 +6403,15 @@ Per-Host Resources: mem-estimate=80.00MB mem-reservation=16.00MB thread-reservat
 01:TOP-N
 |  order by: l_partkey ASC NULLS LAST, l_orderkey DESC
 |  limit with ties: 200
+|  source expr: rank() <= CAST(100 AS BIGINT)
 |  mem-estimate=4.69KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=8 row-size=24B cardinality=200
 |  in pipelines: 01(GETNEXT), 00(OPEN)
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   HDFS partitions=1/1 files=3 size=193.98MB
+   HDFS partitions=1/1 files=3 size=193.99MB
    stored statistics:
-     table: rows=6.00M size=193.98MB
+     table: rows=6.00M size=193.99MB
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=2.14M
    mem-estimate=80.00MB mem-reservation=16.00MB thread-reservation=1
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
index 0d97f7b..bf914a0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q44.test
@@ -47,18 +47,18 @@ WHERE asceding.rnk = descending.rnk
 ORDER BY asceding.rnk
 LIMIT 100
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=26.88MB Threads=7
+Max Per-Host Resource Reservation: Memory=22.75MB Threads=7
 Per-Host Resource Estimates: Memory=306MB
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=306.00MB mem-reservation=26.88MB thread-reservation=7 runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=306.00MB mem-reservation=22.75MB thread-reservation=7 runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  output exprs: rnk, i_product_name, i_product_name
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 23:TOP-N [LIMIT=100]
 |  order by: rnk ASC
-|  mem-estimate=6.65KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=28 row-size=68B cardinality=100
+|  mem-estimate=681B mem-reservation=0B thread-reservation=0
+|  tuple-ids=28 row-size=68B cardinality=10
 |  in pipelines: 23(GETNEXT), 19(OPEN)
 |
 22:HASH JOIN [INNER JOIN]
@@ -66,20 +66,20 @@ PLAN-ROOT SINK
 |  fk/pk conjuncts: i2.i_item_sk = ss_item_sk
 |  runtime filters: RF000[bloom] <- ss_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=27,26,33,32,39,38 row-size=148B cardinality=1.80K
+|  tuple-ids=27,26,33,32,39,38 row-size=148B cardinality=10
 |  in pipelines: 19(GETNEXT), 18(OPEN)
 |
 |--21:HASH JOIN [INNER JOIN]
 |  |  hash predicates: rank() = rank()
 |  |  fk/pk conjuncts: assumed fk/pk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=26,33,32,39,38 row-size=110B cardinality=1.80K
+|  |  tuple-ids=26,33,32,39,38 row-size=110B cardinality=10
 |  |  in pipelines: 18(GETNEXT), 15(OPEN)
 |  |
 |  |--17:SELECT
 |  |  |  predicates: rank() < CAST(11 AS BIGINT)
 |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=39,38 row-size=36B cardinality=1.80K
+|  |  |  tuple-ids=39,38 row-size=36B cardinality=10
 |  |  |  in pipelines: 15(GETNEXT)
 |  |  |
 |  |  16:ANALYTIC
@@ -87,13 +87,15 @@ PLAN-ROOT SINK
 |  |  |  order by: avg(ss_net_profit) DESC
 |  |  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
 |  |  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  |  tuple-ids=39,38 row-size=36B cardinality=17.98K
+|  |  |  tuple-ids=39,38 row-size=36B cardinality=10
 |  |  |  in pipelines: 15(GETNEXT)
 |  |  |
-|  |  15:SORT
+|  |  15:TOP-N
 |  |  |  order by: avg(ss_net_profit) DESC
-|  |  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  |  tuple-ids=39 row-size=28B cardinality=17.98K
+|  |  |  limit with ties: 10
+|  |  |  source expr: rank() < CAST(11 AS BIGINT)
+|  |  |  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=39 row-size=28B cardinality=10
 |  |  |  in pipelines: 15(GETNEXT), 10(OPEN)
 |  |  |
 |  |  14:NESTED LOOP JOIN [INNER JOIN]
@@ -117,10 +119,10 @@ PLAN-ROOT SINK
 |  |  |  |  in pipelines: 12(GETNEXT), 11(OPEN)
 |  |  |  |
 |  |  |  11:SCAN HDFS [tpcds_parquet.store_sales]
-|  |  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |  |     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |  |  |     predicates: ss_addr_sk IS NULL, ss_store_sk = CAST(4 AS INT)
 |  |  |     stored statistics:
-|  |  |       table: rows=2.88M size=200.95MB
+|  |  |       table: rows=2.88M size=196.95MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -138,10 +140,10 @@ PLAN-ROOT SINK
 |  |  |  in pipelines: 10(GETNEXT), 09(OPEN)
 |  |  |
 |  |  09:SCAN HDFS [tpcds_parquet.store_sales ss1]
-|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |  |     predicates: ss_store_sk = CAST(4 AS INT)
 |  |     stored statistics:
-|  |       table: rows=2.88M size=200.95MB
+|  |       table: rows=2.88M size=196.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -156,13 +158,13 @@ PLAN-ROOT SINK
 |  |  fk/pk conjuncts: none
 |  |  runtime filters: RF002[bloom] <- ss_item_sk
 |  |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  |  tuple-ids=26,33,32 row-size=74B cardinality=1.80K
+|  |  tuple-ids=26,33,32 row-size=74B cardinality=10
 |  |  in pipelines: 18(GETNEXT), 06(OPEN)
 |  |
 |  |--08:SELECT
 |  |  |  predicates: rank() < CAST(11 AS BIGINT)
 |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  |  tuple-ids=33,32 row-size=36B cardinality=1.80K
+|  |  |  tuple-ids=33,32 row-size=36B cardinality=10
 |  |  |  in pipelines: 06(GETNEXT)
 |  |  |
 |  |  07:ANALYTIC
@@ -170,13 +172,15 @@ PLAN-ROOT SINK
 |  |  |  order by: avg(ss_net_profit) ASC
 |  |  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
 |  |  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  |  tuple-ids=33,32 row-size=36B cardinality=17.98K
+|  |  |  tuple-ids=33,32 row-size=36B cardinality=10
 |  |  |  in pipelines: 06(GETNEXT)
 |  |  |
-|  |  06:SORT
+|  |  06:TOP-N
 |  |  |  order by: avg(ss_net_profit) ASC
-|  |  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  |  tuple-ids=33 row-size=28B cardinality=17.98K
+|  |  |  limit with ties: 10
+|  |  |  source expr: rank() < CAST(11 AS BIGINT)
+|  |  |  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  |  |  tuple-ids=33 row-size=28B cardinality=10
 |  |  |  in pipelines: 06(GETNEXT), 01(OPEN)
 |  |  |
 |  |  05:NESTED LOOP JOIN [INNER JOIN]
@@ -200,10 +204,10 @@ PLAN-ROOT SINK
 |  |  |  |  in pipelines: 03(GETNEXT), 02(OPEN)
 |  |  |  |
 |  |  |  02:SCAN HDFS [tpcds_parquet.store_sales]
-|  |  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |  |     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |  |  |     predicates: ss_addr_sk IS NULL, ss_store_sk = CAST(4 AS INT)
 |  |  |     stored statistics:
-|  |  |       table: rows=2.88M size=200.95MB
+|  |  |       table: rows=2.88M size=196.95MB
 |  |  |       partitions: 1824/1824 rows=2.88M
 |  |  |       columns: all
 |  |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -221,10 +225,10 @@ PLAN-ROOT SINK
 |  |  |  in pipelines: 01(GETNEXT), 00(OPEN)
 |  |  |
 |  |  00:SCAN HDFS [tpcds_parquet.store_sales ss1]
-|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |  |     predicates: ss_store_sk = CAST(4 AS INT)
 |  |     stored statistics:
-|  |       table: rows=2.88M size=200.95MB
+|  |       table: rows=2.88M size=196.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -256,28 +260,28 @@ PLAN-ROOT SINK
    tuple-ids=27 row-size=38B cardinality=18.00K
    in pipelines: 19(GETNEXT)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=46.56MB Threads=20
-Per-Host Resource Estimates: Memory=364MB
+Max Per-Host Resource Reservation: Memory=34.56MB Threads=20
+Per-Host Resource Estimates: Memory=352MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.52MB mem-reservation=9.81MB thread-reservation=1
+|  Per-Host Resources: mem-estimate=11.23MB mem-reservation=9.81MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: rnk, i_product_name, i_product_name
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 23:TOP-N [LIMIT=100]
 |  order by: rnk ASC
-|  mem-estimate=6.65KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=28 row-size=68B cardinality=100
-|  in pipelines: 23(GETNEXT), 06(OPEN)
+|  mem-estimate=681B mem-reservation=0B thread-reservation=0
+|  tuple-ids=28 row-size=68B cardinality=10
+|  in pipelines: 23(GETNEXT), 31(OPEN)
 |
 22:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: ss_item_sk = i2.i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i2.i_item_sk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=33,32,26,39,38,27 row-size=148B cardinality=1.80K
-|  in pipelines: 06(GETNEXT), 19(OPEN)
+|  tuple-ids=33,32,26,39,38,27 row-size=148B cardinality=10
+|  in pipelines: 31(GETNEXT), 19(OPEN)
 |
-|--40:EXCHANGE [UNPARTITIONED]
+|--42:EXCHANGE [UNPARTITIONED]
 |  |  mem-estimate=710.69KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=27 row-size=38B cardinality=18.00K
 |  |  in pipelines: 19(GETNEXT)
@@ -298,51 +302,59 @@ PLAN-ROOT SINK
 |  hash predicates: rank() = rank()
 |  fk/pk conjuncts: assumed fk/pk
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=33,32,26,39,38 row-size=110B cardinality=1.80K
-|  in pipelines: 06(GETNEXT), 15(OPEN)
+|  tuple-ids=33,32,26,39,38 row-size=110B cardinality=10
+|  in pipelines: 31(GETNEXT), 40(OPEN)
 |
-|--39:EXCHANGE [UNPARTITIONED]
-|  |  mem-estimate=65.07KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=39,38 row-size=36B cardinality=1.80K
-|  |  in pipelines: 15(GETNEXT)
+|--41:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=39,38 row-size=36B cardinality=10
+|  |  in pipelines: 40(GETNEXT)
 |  |
 |  F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=4.25MB mem-reservation=4.00MB thread-reservation=1
+|  Per-Host Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 |  17:SELECT
 |  |  predicates: rank() < CAST(11 AS BIGINT)
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=39,38 row-size=36B cardinality=1.80K
-|  |  in pipelines: 15(GETNEXT)
+|  |  tuple-ids=39,38 row-size=36B cardinality=10
+|  |  in pipelines: 40(GETNEXT)
 |  |
 |  16:ANALYTIC
 |  |  functions: rank()
 |  |  order by: avg(ss_net_profit) DESC
 |  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
 |  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=39,38 row-size=36B cardinality=17.98K
-|  |  in pipelines: 15(GETNEXT)
+|  |  tuple-ids=39,38 row-size=36B cardinality=10
+|  |  in pipelines: 40(GETNEXT)
 |  |
-|  38:MERGING-EXCHANGE [UNPARTITIONED]
+|  40:TOP-N
 |  |  order by: avg(ss_net_profit) DESC
-|  |  mem-estimate=259.83KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=39 row-size=28B cardinality=17.98K
+|  |  limit with ties: 10
+|  |  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=39 row-size=28B cardinality=10
+|  |  in pipelines: 40(GETNEXT), 15(OPEN)
+|  |
+|  39:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=39 row-size=28B cardinality=10
 |  |  in pipelines: 15(GETNEXT)
 |  |
 |  F08:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=16.02MB mem-reservation=7.94MB thread-reservation=1
-|  15:SORT
+|  Per-Host Resources: mem-estimate=10.17MB mem-reservation=1.94MB thread-reservation=1
+|  15:TOP-N
 |  |  order by: avg(ss_net_profit) DESC
-|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=39 row-size=28B cardinality=17.98K
-|  |  in pipelines: 15(GETNEXT), 33(OPEN)
+|  |  limit with ties: 10
+|  |  source expr: rank() < CAST(11 AS BIGINT)
+|  |  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=39 row-size=28B cardinality=10
+|  |  in pipelines: 15(GETNEXT), 34(OPEN)
 |  |
 |  14:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  |  predicates: avg(ss_net_profit) > CAST(0.9 AS DECIMAL(1,1)) * avg(ss_net_profit)
 |  |  mem-estimate=12B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=15,19 row-size=28B cardinality=17.98K
-|  |  in pipelines: 33(GETNEXT), 13(OPEN)
+|  |  in pipelines: 34(GETNEXT), 13(OPEN)
 |  |
-|  |--37:EXCHANGE [BROADCAST]
+|  |--38:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=19 row-size=12B cardinality=1
 |  |  |  in pipelines: 13(GETNEXT)
@@ -353,25 +365,25 @@ PLAN-ROOT SINK
 |  |  |  limit: 1
 |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=19 row-size=12B cardinality=1
-|  |  |  in pipelines: 13(GETNEXT), 35(OPEN)
+|  |  |  in pipelines: 13(GETNEXT), 36(OPEN)
 |  |  |
-|  |  36:EXCHANGE [UNPARTITIONED]
+|  |  37:EXCHANGE [UNPARTITIONED]
 |  |  |  limit: 2
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=19 row-size=12B cardinality=2
-|  |  |  in pipelines: 35(GETNEXT)
+|  |  |  in pipelines: 36(GETNEXT)
 |  |  |
 |  |  F10:PLAN FRAGMENT [HASH(ss_store_sk)] hosts=3 instances=3
 |  |  Per-Host Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
-|  |  35:AGGREGATE [FINALIZE]
+|  |  36:AGGREGATE [FINALIZE]
 |  |  |  output: avg:merge(ss_net_profit)
 |  |  |  group by: ss_store_sk
 |  |  |  limit: 2
 |  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  |  tuple-ids=19 row-size=12B cardinality=2
-|  |  |  in pipelines: 35(GETNEXT), 11(OPEN)
+|  |  |  in pipelines: 36(GETNEXT), 11(OPEN)
 |  |  |
-|  |  34:EXCHANGE [HASH(ss_store_sk)]
+|  |  35:EXCHANGE [HASH(ss_store_sk)]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=18 row-size=12B cardinality=6
 |  |  |  in pipelines: 11(GETNEXT)
@@ -386,10 +398,10 @@ PLAN-ROOT SINK
 |  |  |  in pipelines: 11(GETNEXT)
 |  |  |
 |  |  11:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |  |     predicates: ss_addr_sk IS NULL, ss_store_sk = CAST(4 AS INT)
 |  |     stored statistics:
-|  |       table: rows=2.88M size=200.95MB
+|  |       table: rows=2.88M size=196.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -399,14 +411,14 @@ PLAN-ROOT SINK
 |  |     tuple-ids=17 row-size=12B cardinality=53.06K
 |  |     in pipelines: 11(GETNEXT)
 |  |
-|  33:AGGREGATE [FINALIZE]
+|  34:AGGREGATE [FINALIZE]
 |  |  output: avg:merge(ss_net_profit)
 |  |  group by: ss_item_sk
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=15 row-size=16B cardinality=17.98K
-|  |  in pipelines: 33(GETNEXT), 09(OPEN)
+|  |  in pipelines: 34(GETNEXT), 09(OPEN)
 |  |
-|  32:EXCHANGE [HASH(ss_item_sk)]
+|  33:EXCHANGE [HASH(ss_item_sk)]
 |  |  mem-estimate=153.62KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=16B cardinality=17.98K
 |  |  in pipelines: 09(GETNEXT)
@@ -421,10 +433,10 @@ PLAN-ROOT SINK
 |  |  in pipelines: 09(GETNEXT)
 |  |
 |  09:SCAN HDFS [tpcds_parquet.store_sales ss1, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |     predicates: ss_store_sk = CAST(4 AS INT)
 |     stored statistics:
-|       table: rows=2.88M size=200.95MB
+|       table: rows=2.88M size=196.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -438,10 +450,10 @@ PLAN-ROOT SINK
 |  hash predicates: ss_item_sk = i1.i_item_sk
 |  fk/pk conjuncts: none
 |  mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=33,32,26 row-size=74B cardinality=1.80K
-|  in pipelines: 06(GETNEXT), 18(OPEN)
+|  tuple-ids=33,32,26 row-size=74B cardinality=10
+|  in pipelines: 31(GETNEXT), 18(OPEN)
 |
-|--31:EXCHANGE [UNPARTITIONED]
+|--32:EXCHANGE [UNPARTITIONED]
 |  |  mem-estimate=710.69KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=26 row-size=38B cardinality=18.00K
 |  |  in pipelines: 18(GETNEXT)
@@ -461,29 +473,37 @@ PLAN-ROOT SINK
 08:SELECT
 |  predicates: rank() < CAST(11 AS BIGINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=33,32 row-size=36B cardinality=1.80K
-|  in pipelines: 06(GETNEXT)
+|  tuple-ids=33,32 row-size=36B cardinality=10
+|  in pipelines: 31(GETNEXT)
 |
 07:ANALYTIC
 |  functions: rank()
 |  order by: avg(ss_net_profit) ASC
 |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=33,32 row-size=36B cardinality=17.98K
-|  in pipelines: 06(GETNEXT)
+|  tuple-ids=33,32 row-size=36B cardinality=10
+|  in pipelines: 31(GETNEXT)
 |
-30:MERGING-EXCHANGE [UNPARTITIONED]
+31:TOP-N
 |  order by: avg(ss_net_profit) ASC
-|  mem-estimate=259.83KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=33 row-size=28B cardinality=17.98K
+|  limit with ties: 10
+|  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  tuple-ids=33 row-size=28B cardinality=10
+|  in pipelines: 31(GETNEXT), 06(OPEN)
+|
+30:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=33 row-size=28B cardinality=10
 |  in pipelines: 06(GETNEXT)
 |
 F01:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3
-Per-Host Resources: mem-estimate=16.02MB mem-reservation=7.94MB thread-reservation=1
-06:SORT
+Per-Host Resources: mem-estimate=10.17MB mem-reservation=1.94MB thread-reservation=1
+06:TOP-N
 |  order by: avg(ss_net_profit) ASC
-|  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=33 row-size=28B cardinality=17.98K
+|  limit with ties: 10
+|  source expr: rank() < CAST(11 AS BIGINT)
+|  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  tuple-ids=33 row-size=28B cardinality=10
 |  in pipelines: 06(GETNEXT), 25(OPEN)
 |
 05:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -536,10 +556,10 @@ Per-Host Resources: mem-estimate=16.02MB mem-reservation=7.94MB thread-reservati
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |     predicates: ss_addr_sk IS NULL, ss_store_sk = CAST(4 AS INT)
 |     stored statistics:
-|       table: rows=2.88M size=200.95MB
+|       table: rows=2.88M size=196.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -571,10 +591,10 @@ Per-Host Resources: mem-estimate=58.00MB mem-reservation=3.00MB thread-reservati
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales ss1, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.95MB
+   HDFS partitions=1824/1824 files=1824 size=196.95MB
    predicates: ss_store_sk = CAST(4 AS INT)
    stored statistics:
-     table: rows=2.88M size=200.95MB
+     table: rows=2.88M size=196.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -584,27 +604,27 @@ Per-Host Resources: mem-estimate=58.00MB mem-reservation=3.00MB thread-reservati
    tuple-ids=0 row-size=16B cardinality=480.07K
    in pipelines: 00(GETNEXT)
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=84.12MB Threads=27
-Per-Host Resource Estimates: Memory=366MB
+Max Per-Host Resource Reservation: Memory=60.12MB Threads=27
+Per-Host Resource Estimates: Memory=342MB
 F05:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=4.35MB mem-reservation=4.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 PLAN-ROOT SINK
 |  output exprs: rnk, i_product_name, i_product_name
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |
 23:TOP-N [LIMIT=100]
 |  order by: rnk ASC
-|  mem-estimate=6.65KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=28 row-size=68B cardinality=100
-|  in pipelines: 23(GETNEXT), 06(OPEN)
+|  mem-estimate=681B mem-reservation=0B thread-reservation=0
+|  tuple-ids=28 row-size=68B cardinality=10
+|  in pipelines: 23(GETNEXT), 31(OPEN)
 |
 22:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: ss_item_sk = i2.i_item_sk
 |  fk/pk conjuncts: ss_item_sk = i2.i_item_sk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=33,32,26,39,38,27 row-size=148B cardinality=1.80K
-|  in pipelines: 06(GETNEXT), 19(OPEN)
+|  tuple-ids=33,32,26,39,38,27 row-size=148B cardinality=10
+|  in pipelines: 31(GETNEXT), 19(OPEN)
 |
 |--F14:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=4.57MB mem-reservation=3.88MB thread-reservation=1
@@ -613,7 +633,7 @@ PLAN-ROOT SINK
 |  |  build expressions: i2.i_item_sk
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  40:EXCHANGE [UNPARTITIONED]
+|  42:EXCHANGE [UNPARTITIONED]
 |  |  mem-estimate=710.69KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=27 row-size=38B cardinality=18.00K
 |  |  in pipelines: 19(GETNEXT)
@@ -635,57 +655,65 @@ PLAN-ROOT SINK
 |  hash predicates: rank() = rank()
 |  fk/pk conjuncts: assumed fk/pk
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=33,32,26,39,38 row-size=110B cardinality=1.80K
-|  in pipelines: 06(GETNEXT), 15(OPEN)
+|  tuple-ids=33,32,26,39,38 row-size=110B cardinality=10
+|  in pipelines: 31(GETNEXT), 40(OPEN)
 |
 |--F15:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  |  Per-Instance Resources: mem-estimate=3.94MB mem-reservation=3.88MB thread-reservation=1
+|  |  Per-Instance Resources: mem-estimate=3.89MB mem-reservation=3.88MB thread-reservation=1
 |  JOIN BUILD
 |  |  join-table-id=01 plan-id=02 cohort-id=01
 |  |  build expressions: rank()
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  39:EXCHANGE [UNPARTITIONED]
-|  |  mem-estimate=65.07KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=39,38 row-size=36B cardinality=1.80K
-|  |  in pipelines: 15(GETNEXT)
+|  41:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=39,38 row-size=36B cardinality=10
+|  |  in pipelines: 40(GETNEXT)
 |  |
 |  F12:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Instance Resources: mem-estimate=4.35MB mem-reservation=4.00MB thread-reservation=1
+|  Per-Instance Resources: mem-estimate=4.02MB mem-reservation=4.00MB thread-reservation=1
 |  17:SELECT
 |  |  predicates: rank() < CAST(11 AS BIGINT)
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=39,38 row-size=36B cardinality=1.80K
-|  |  in pipelines: 15(GETNEXT)
+|  |  tuple-ids=39,38 row-size=36B cardinality=10
+|  |  in pipelines: 40(GETNEXT)
 |  |
 |  16:ANALYTIC
 |  |  functions: rank()
 |  |  order by: avg(ss_net_profit) DESC
 |  |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
 |  |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=39,38 row-size=36B cardinality=17.98K
-|  |  in pipelines: 15(GETNEXT)
+|  |  tuple-ids=39,38 row-size=36B cardinality=10
+|  |  in pipelines: 40(GETNEXT)
 |  |
-|  38:MERGING-EXCHANGE [UNPARTITIONED]
+|  40:TOP-N
 |  |  order by: avg(ss_net_profit) DESC
-|  |  mem-estimate=355.83KB mem-reservation=0B thread-reservation=0
-|  |  tuple-ids=39 row-size=28B cardinality=17.98K
+|  |  limit with ties: 10
+|  |  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=39 row-size=28B cardinality=10
+|  |  in pipelines: 40(GETNEXT), 15(OPEN)
+|  |
+|  39:EXCHANGE [UNPARTITIONED]
+|  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=39 row-size=28B cardinality=10
 |  |  in pipelines: 15(GETNEXT)
 |  |
 |  F08:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=6
-|  Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservation=1
-|  15:SORT
+|  Per-Instance Resources: mem-estimate=10.21MB mem-reservation=1.94MB thread-reservation=1
+|  15:TOP-N
 |  |  order by: avg(ss_net_profit) DESC
-|  |  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
-|  |  tuple-ids=39 row-size=28B cardinality=17.98K
-|  |  in pipelines: 15(GETNEXT), 33(OPEN)
+|  |  limit with ties: 10
+|  |  source expr: rank() < CAST(11 AS BIGINT)
+|  |  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  |  tuple-ids=39 row-size=28B cardinality=10
+|  |  in pipelines: 15(GETNEXT), 34(OPEN)
 |  |
 |  14:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
 |  |  join table id: 02
 |  |  predicates: avg(ss_net_profit) > CAST(0.9 AS DECIMAL(1,1)) * avg(ss_net_profit)
 |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=15,19 row-size=28B cardinality=17.98K
-|  |  in pipelines: 33(GETNEXT), 13(OPEN)
+|  |  in pipelines: 34(GETNEXT), 13(OPEN)
 |  |
 |  |--F16:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=3
 |  |  |  Per-Instance Resources: mem-estimate=16.01KB mem-reservation=0B thread-reservation=1
@@ -693,7 +721,7 @@ PLAN-ROOT SINK
 |  |  |  join-table-id=02 plan-id=03 cohort-id=02
 |  |  |  mem-estimate=12B mem-reservation=0B thread-reservation=0
 |  |  |
-|  |  37:EXCHANGE [BROADCAST]
+|  |  38:EXCHANGE [BROADCAST]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=19 row-size=12B cardinality=1
 |  |  |  in pipelines: 13(GETNEXT)
@@ -704,25 +732,25 @@ PLAN-ROOT SINK
 |  |  |  limit: 1
 |  |  |  mem-estimate=0B mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=19 row-size=12B cardinality=1
-|  |  |  in pipelines: 13(GETNEXT), 35(OPEN)
+|  |  |  in pipelines: 13(GETNEXT), 36(OPEN)
 |  |  |
-|  |  36:EXCHANGE [UNPARTITIONED]
+|  |  37:EXCHANGE [UNPARTITIONED]
 |  |  |  limit: 2
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=19 row-size=12B cardinality=2
-|  |  |  in pipelines: 35(GETNEXT)
+|  |  |  in pipelines: 36(GETNEXT)
 |  |  |
 |  |  F10:PLAN FRAGMENT [HASH(ss_store_sk)] hosts=3 instances=6
 |  |  Per-Instance Resources: mem-estimate=10.02MB mem-reservation=1.94MB thread-reservation=1
-|  |  35:AGGREGATE [FINALIZE]
+|  |  36:AGGREGATE [FINALIZE]
 |  |  |  output: avg:merge(ss_net_profit)
 |  |  |  group by: ss_store_sk
 |  |  |  limit: 2
 |  |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  |  tuple-ids=19 row-size=12B cardinality=2
-|  |  |  in pipelines: 35(GETNEXT), 11(OPEN)
+|  |  |  in pipelines: 36(GETNEXT), 11(OPEN)
 |  |  |
-|  |  34:EXCHANGE [HASH(ss_store_sk)]
+|  |  35:EXCHANGE [HASH(ss_store_sk)]
 |  |  |  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
 |  |  |  tuple-ids=18 row-size=12B cardinality=6
 |  |  |  in pipelines: 11(GETNEXT)
@@ -737,10 +765,10 @@ PLAN-ROOT SINK
 |  |  |  in pipelines: 11(GETNEXT)
 |  |  |
 |  |  11:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|  |     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|  |     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |  |     predicates: ss_addr_sk IS NULL, ss_store_sk = CAST(4 AS INT)
 |  |     stored statistics:
-|  |       table: rows=2.88M size=200.95MB
+|  |       table: rows=2.88M size=196.95MB
 |  |       partitions: 1824/1824 rows=2.88M
 |  |       columns: all
 |  |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -750,14 +778,14 @@ PLAN-ROOT SINK
 |  |     tuple-ids=17 row-size=12B cardinality=53.06K
 |  |     in pipelines: 11(GETNEXT)
 |  |
-|  33:AGGREGATE [FINALIZE]
+|  34:AGGREGATE [FINALIZE]
 |  |  output: avg:merge(ss_net_profit)
 |  |  group by: ss_item_sk
 |  |  mem-estimate=10.00MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
 |  |  tuple-ids=15 row-size=16B cardinality=17.98K
-|  |  in pipelines: 33(GETNEXT), 09(OPEN)
+|  |  in pipelines: 34(GETNEXT), 09(OPEN)
 |  |
-|  32:EXCHANGE [HASH(ss_item_sk)]
+|  33:EXCHANGE [HASH(ss_item_sk)]
 |  |  mem-estimate=213.62KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=14 row-size=16B cardinality=17.98K
 |  |  in pipelines: 09(GETNEXT)
@@ -772,10 +800,10 @@ PLAN-ROOT SINK
 |  |  in pipelines: 09(GETNEXT)
 |  |
 |  09:SCAN HDFS [tpcds_parquet.store_sales ss1, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |     predicates: ss_store_sk = CAST(4 AS INT)
 |     stored statistics:
-|       table: rows=2.88M size=200.95MB
+|       table: rows=2.88M size=196.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -790,8 +818,8 @@ PLAN-ROOT SINK
 |  hash predicates: ss_item_sk = i1.i_item_sk
 |  fk/pk conjuncts: none
 |  mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
-|  tuple-ids=33,32,26 row-size=74B cardinality=1.80K
-|  in pipelines: 06(GETNEXT), 18(OPEN)
+|  tuple-ids=33,32,26 row-size=74B cardinality=10
+|  in pipelines: 31(GETNEXT), 18(OPEN)
 |
 |--F17:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  |  Per-Instance Resources: mem-estimate=4.57MB mem-reservation=3.88MB thread-reservation=1
@@ -800,7 +828,7 @@ PLAN-ROOT SINK
 |  |  build expressions: i1.i_item_sk
 |  |  mem-estimate=3.88MB mem-reservation=3.88MB spill-buffer=64.00KB thread-reservation=0
 |  |
-|  31:EXCHANGE [UNPARTITIONED]
+|  32:EXCHANGE [UNPARTITIONED]
 |  |  mem-estimate=710.69KB mem-reservation=0B thread-reservation=0
 |  |  tuple-ids=26 row-size=38B cardinality=18.00K
 |  |  in pipelines: 18(GETNEXT)
@@ -820,29 +848,37 @@ PLAN-ROOT SINK
 08:SELECT
 |  predicates: rank() < CAST(11 AS BIGINT)
 |  mem-estimate=0B mem-reservation=0B thread-reservation=0
-|  tuple-ids=33,32 row-size=36B cardinality=1.80K
-|  in pipelines: 06(GETNEXT)
+|  tuple-ids=33,32 row-size=36B cardinality=10
+|  in pipelines: 31(GETNEXT)
 |
 07:ANALYTIC
 |  functions: rank()
 |  order by: avg(ss_net_profit) ASC
 |  window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
 |  mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=33,32 row-size=36B cardinality=17.98K
-|  in pipelines: 06(GETNEXT)
+|  tuple-ids=33,32 row-size=36B cardinality=10
+|  in pipelines: 31(GETNEXT)
 |
-30:MERGING-EXCHANGE [UNPARTITIONED]
+31:TOP-N
 |  order by: avg(ss_net_profit) ASC
-|  mem-estimate=355.83KB mem-reservation=0B thread-reservation=0
-|  tuple-ids=33 row-size=28B cardinality=17.98K
+|  limit with ties: 10
+|  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  tuple-ids=33 row-size=28B cardinality=10
+|  in pipelines: 31(GETNEXT), 06(OPEN)
+|
+30:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
+|  tuple-ids=33 row-size=28B cardinality=10
 |  in pipelines: 06(GETNEXT)
 |
 F01:PLAN FRAGMENT [HASH(ss_item_sk)] hosts=3 instances=6
-Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reservation=1
-06:SORT
+Per-Instance Resources: mem-estimate=10.21MB mem-reservation=1.94MB thread-reservation=1
+06:TOP-N
 |  order by: avg(ss_net_profit) ASC
-|  mem-estimate=6.00MB mem-reservation=6.00MB spill-buffer=2.00MB thread-reservation=0
-|  tuple-ids=33 row-size=28B cardinality=17.98K
+|  limit with ties: 10
+|  source expr: rank() < CAST(11 AS BIGINT)
+|  mem-estimate=280B mem-reservation=0B thread-reservation=0
+|  tuple-ids=33 row-size=28B cardinality=10
 |  in pipelines: 06(GETNEXT), 25(OPEN)
 |
 05:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -902,10 +938,10 @@ Per-Instance Resources: mem-estimate=16.00MB mem-reservation=7.94MB thread-reser
 |  |  in pipelines: 02(GETNEXT)
 |  |
 |  02:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |     predicates: ss_addr_sk IS NULL, ss_store_sk = CAST(4 AS INT)
 |     stored statistics:
-|       table: rows=2.88M size=200.95MB
+|       table: rows=2.88M size=196.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -937,10 +973,10 @@ Per-Instance Resources: mem-estimate=26.00MB mem-reservation=3.00MB thread-reser
 |  in pipelines: 00(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales ss1, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.95MB
+   HDFS partitions=1824/1824 files=1824 size=196.95MB
    predicates: ss_store_sk = CAST(4 AS INT)
    stored statistics:
-     table: rows=2.88M size=200.95MB
+     table: rows=2.88M size=196.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
index bc8328f..ef6bc6c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q67.test
@@ -73,6 +73,7 @@ PLAN-ROOT SINK
 09:TOP-N
 |  order by: CASE valid_tid(4,5,6,7,8,9,10,11,12) WHEN 4 THEN i_category WHEN 5 THEN i_category WHEN 6 THEN i_category WHEN 7 THEN i_category WHEN 8 THEN i_category WHEN 9 THEN i_category WHEN 10 THEN i_category WHEN 11 THEN i_category WHEN 12 THEN NULL END ASC NULLS LAST, aggif(valid_tid(4,5,6,7,8,9,10,11,12) IN (4, 5, 6, 7, 8, 9, 10, 11, 12), CASE valid_tid(4,5,6,7,8,9,10,11,12) WHEN 4 THEN sum(coalesce(ss_sales_price * ss_quantity, 0)) WHEN 5 THEN sum(coalesce(ss_sales_price * ss_quan [...]
 |  limit with ties: 200
+|  source expr: rank() <= CAST(100 AS BIGINT)
 |  mem-estimate=17.97KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=25 row-size=92B cardinality=200
 |  in pipelines: 09(GETNEXT), 08(OPEN)
@@ -240,6 +241,7 @@ Per-Host Resources: mem-estimate=43.90GB mem-reservation=244.75MB thread-reserva
 09:TOP-N
 |  order by: CASE valid_tid(4,5,6,7,8,9,10,11,12) WHEN 4 THEN i_category WHEN 5 THEN i_category WHEN 6 THEN i_category WHEN 7 THEN i_category WHEN 8 THEN i_category WHEN 9 THEN i_category WHEN 10 THEN i_category WHEN 11 THEN i_category WHEN 12 THEN NULL END ASC NULLS LAST, aggif(valid_tid(4,5,6,7,8,9,10,11,12) IN (4, 5, 6, 7, 8, 9, 10, 11, 12), CASE valid_tid(4,5,6,7,8,9,10,11,12) WHEN 4 THEN sum(coalesce(ss_sales_price * ss_quantity, 0)) WHEN 5 THEN sum(coalesce(ss_sales_price * ss_quan [...]
 |  limit with ties: 200
+|  source expr: rank() <= CAST(100 AS BIGINT)
 |  mem-estimate=17.97KB mem-reservation=0B thread-reservation=0
 |  tuple-ids=25 row-size=92B cardinality=200
 |  in pipelines: 09(GETNEXT), 08(OPEN)
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
index b873aff..9f57b07 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds/tpcds-q70.test
@@ -109,8 +109,11 @@ PLAN-ROOT SINK
 |  |  tuple-ids=26,25 row-size=38B cardinality=1
 |  |  in pipelines: 09(GETNEXT)
 |  |
-|  09:SORT
-|  |  order by: s_state ASC NULLS LAST, sum(ss_net_profit) DESC
+|  09:TOP-N
+|  |  partition by: s_state
+|  |  order by: sum(ss_net_profit) DESC
+|  |  partition limit: 5 (include ties)
+|  |  source expr: rank() <= CAST(5 AS BIGINT)
 |  |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
 |  |  tuple-ids=26 row-size=30B cardinality=1
 |  |  in pipelines: 09(GETNEXT), 08(OPEN)
@@ -162,10 +165,10 @@ PLAN-ROOT SINK
 |  |     in pipelines: 04(GETNEXT)
 |  |
 |  03:SCAN HDFS [tpcds_parquet.store_sales]
-|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |     runtime filters: RF006[bloom] -> ss_sold_date_sk, RF008[bloom] -> ss_store_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.95MB
+|       table: rows=2.88M size=196.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -214,10 +217,10 @@ PLAN-ROOT SINK
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales]
-   HDFS partitions=1824/1824 files=1824 size=200.95MB
+   HDFS partitions=1824/1824 files=1824 size=196.95MB
    runtime filters: RF002[bloom] -> ss_store_sk, RF004[bloom] -> ss_sold_date_sk
    stored statistics:
-     table: rows=2.88M size=200.95MB
+     table: rows=2.88M size=196.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -341,8 +344,11 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat
 |  |  tuple-ids=26,25 row-size=38B cardinality=1
 |  |  in pipelines: 09(GETNEXT)
 |  |
-|  09:SORT
-|  |  order by: s_state ASC NULLS LAST, sum(ss_net_profit) DESC
+|  09:TOP-N
+|  |  partition by: s_state
+|  |  order by: sum(ss_net_profit) DESC
+|  |  partition limit: 5 (include ties)
+|  |  source expr: rank() <= CAST(5 AS BIGINT)
 |  |  mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
 |  |  tuple-ids=26 row-size=30B cardinality=1
 |  |  in pipelines: 09(GETNEXT), 25(OPEN)
@@ -422,10 +428,10 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat
 |  |     in pipelines: 04(GETNEXT)
 |  |
 |  03:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-|     HDFS partitions=1824/1824 files=1824 size=200.95MB
+|     HDFS partitions=1824/1824 files=1824 size=196.95MB
 |     runtime filters: RF006[bloom] -> ss_sold_date_sk, RF008[bloom] -> ss_store_sk
 |     stored statistics:
-|       table: rows=2.88M size=200.95MB
+|       table: rows=2.88M size=196.95MB
 |       partitions: 1824/1824 rows=2.88M
 |       columns: all
 |     extrapolated-rows=disabled max-scan-range-rows=130.09K
@@ -488,10 +494,10 @@ Per-Host Resources: mem-estimate=70.91MB mem-reservation=15.81MB thread-reservat
 |     in pipelines: 01(GETNEXT)
 |
 00:SCAN HDFS [tpcds_parquet.store_sales, RANDOM]
-   HDFS partitions=1824/1824 files=1824 size=200.95MB
+   HDFS partitions=1824/1824 files=1824 size=196.95MB
    runtime filters: RF002[bloom] -> ss_store_sk, RF004[bloom] -> ss_sold_date_sk
    stored statistics:
-     table: rows=2.88M size=200.95MB
+     table: rows=2.88M size=196.95MB
      partitions: 1824/1824 rows=2.88M
      columns: all
    extrapolated-rows=disabled max-scan-range-rows=130.09K
diff --git a/testdata/workloads/functional-query/queries/QueryTest/analytic-fns-tpcds-partitioned-topn.test b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns-tpcds-partitioned-topn.test
new file mode 100644
index 0000000..87b1110
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns-tpcds-partitioned-topn.test
@@ -0,0 +1,325 @@
+====
+---- QUERY
+# Partitioned Top-N over a large input data set. Uses rank so has some
+# ties. Most expensive items per store.
+select *
+from (
+  select s_store_name, s_state, ss_list_price,
+        rank() over (partition by s_store_name, s_state order by ss_list_price desc nulls last) rnk
+  from store_sales ss
+      join store s on ss_store_sk = s_store_sk) v
+where rnk <= 5;
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+'ation','TN',200.00,1
+'ation','TN',199.98,2
+'ation','TN',199.98,2
+'ation','TN',199.92,4
+'ation','TN',199.92,4
+'eing','TN',199.98,1
+'eing','TN',199.96,2
+'eing','TN',199.94,3
+'eing','TN',199.88,4
+'eing','TN',199.86,5
+'eing','TN',199.86,5
+'bar','TN',199.94,1
+'bar','TN',199.90,2
+'bar','TN',199.90,2
+'bar','TN',199.88,4
+'bar','TN',199.86,5
+'able','TN',199.96,1
+'able','TN',199.96,1
+'able','TN',199.94,3
+'able','TN',199.90,4
+'able','TN',199.86,5
+'ese','TN',199.88,1
+'ese','TN',199.84,2
+'ese','TN',199.74,3
+'ese','TN',199.70,4
+'ese','TN',199.68,5
+'ought','TN',199.94,1
+'ought','TN',199.94,1
+'ought','TN',199.84,3
+'ought','TN',199.80,4
+'ought','TN',199.48,5
+'ought','TN',199.48,5
+---- TYPES
+STRING, STRING, DECIMAL, BIGINT
+====
+---- QUERY
+# Partitioned Top-N over a large input data set. Uses row_number so has no
+# ties. Most expensive items per store.
+select *
+from (
+  select s_store_name, s_state, ss_list_price,
+        row_number() over (partition by s_store_name, s_state order by ss_list_price desc nulls last) rn
+  from store_sales ss
+      join store s on ss_store_sk = s_store_sk) v
+where rn <= 5;
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+'able','TN',199.96,1
+'able','TN',199.96,2
+'able','TN',199.94,3
+'able','TN',199.90,4
+'able','TN',199.86,5
+'eing','TN',199.98,1
+'eing','TN',199.96,2
+'eing','TN',199.94,3
+'eing','TN',199.88,4
+'eing','TN',199.86,5
+'ese','TN',199.88,1
+'ese','TN',199.84,2
+'ese','TN',199.74,3
+'ese','TN',199.70,4
+'ese','TN',199.68,5
+'ought','TN',199.94,1
+'ought','TN',199.94,2
+'ought','TN',199.84,3
+'ought','TN',199.80,4
+'ought','TN',199.48,5
+'ation','TN',200.00,1
+'ation','TN',199.98,2
+'ation','TN',199.98,3
+'ation','TN',199.92,4
+'ation','TN',199.92,5
+'bar','TN',199.94,1
+'bar','TN',199.90,2
+'bar','TN',199.90,3
+'bar','TN',199.88,4
+'bar','TN',199.86,5
+---- TYPES
+STRING, STRING, DECIMAL, BIGINT
+====
+---- QUERY
+# Same with nulls first results in a huge number of ties for first.
+select s_store_name, s_state, ss_list_price, count(*)
+from (
+  select s_store_name, s_state, ss_list_price,
+        rank() over (partition by s_store_name, s_state order by ss_list_price desc) rnk
+  from store_sales ss
+      join store s on ss_store_sk = s_store_sk) v
+where rnk <= 5
+group by s_store_name, s_state, ss_list_price
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+'ese','TN',NULL,10794
+'able','TN',NULL,10874
+'ought','TN',NULL,10715
+'bar','TN',NULL,10955
+'eing','TN',NULL,10917
+'ation','TN',NULL,10826
+---- TYPES
+STRING, STRING, DECIMAL, BIGINT
+====
+---- QUERY
+# Same with row_number() assigns 1, 2, 3, 4, 5 to the ties.
+select s_store_name, s_state, ss_list_price, rn
+from (
+  select s_store_name, s_state, ss_list_price,
+        row_number() over (partition by s_store_name, s_state order by ss_list_price desc) rn
+  from store_sales ss
+      join store s on ss_store_sk = s_store_sk) v
+where rn <= 5
+---- RESULTS: VERIFY_IS_EQUAL_SORTED
+'ese','TN',NULL,1
+'ese','TN',NULL,2
+'ese','TN',NULL,3
+'ese','TN',NULL,4
+'ese','TN',NULL,5
+'ation','TN',NULL,1
+'ation','TN',NULL,2
+'ation','TN',NULL,3
+'ation','TN',NULL,4
+'ation','TN',NULL,5
+'bar','TN',NULL,1
+'bar','TN',NULL,2
+'bar','TN',NULL,3
+'bar','TN',NULL,4
+'bar','TN',NULL,5
+'able','TN',NULL,1
+'able','TN',NULL,2
+'able','TN',NULL,3
+'able','TN',NULL,4
+'able','TN',NULL,5
+'eing','TN',NULL,1
+'eing','TN',NULL,2
+'eing','TN',NULL,3
+'eing','TN',NULL,4
+'eing','TN',NULL,5
+'ought','TN',NULL,1
+'ought','TN',NULL,2
+'ought','TN',NULL,3
+'ought','TN',NULL,4
+'ought','TN',NULL,5
+---- TYPES
+STRING, STRING, DECIMAL, BIGINT
+====
+---- QUERY
+# Partitioned Top-N with higher number of partitions -
+# ss_sold_date_sk has ~1800 distinct values.
+# Find the 100th most expensive sale from every day.
+select * from (
+  select d_date, i_item_id, ss_list_price,
+        rank() over (partition by d_date order by ss_list_price desc) rnk
+  from store_sales ss
+      join item i on ss_item_sk = i_item_sk
+      join date_dim d on ss_sold_date_sk = d_date_sk
+  where ss_list_price is not null) v
+where rnk = 100
+order by d_date, i_item_id
+limit 50
+---- RESULTS
+'1998-01-02','AAAAAAAAAELBAAAA',141.72,100
+'1998-01-03','AAAAAAAABHIDAAAA',136.09,100
+'1998-01-04','AAAAAAAAAICAAAAA',139.69,100
+'1998-01-05','AAAAAAAAOMCAAAAA',133.78,100
+'1998-01-06','AAAAAAAACEFAAAAA',132.41,100
+'1998-01-07','AAAAAAAAIGGCAAAA',133.14,100
+'1998-01-08','AAAAAAAAGIIBAAAA',140.87,100
+'1998-01-09','AAAAAAAAGBCBAAAA',133.55,100
+'1998-01-10','AAAAAAAAOCCEAAAA',135.58,100
+'1998-01-11','AAAAAAAAIHEAAAAA',135.73,100
+'1998-01-12','AAAAAAAAJFIAAAAA',139.15,100
+'1998-01-13','AAAAAAAACCBDAAAA',128.56,100
+'1998-01-14','AAAAAAAAFMCDAAAA',136.40,100
+'1998-01-16','AAAAAAAALNGBAAAA',141.87,100
+'1998-01-17','AAAAAAAAHDBCAAAA',127.54,100
+'1998-01-18','AAAAAAAAPHLBAAAA',140.04,100
+'1998-01-19','AAAAAAAACIJBAAAA',131.46,100
+'1998-01-20','AAAAAAAAIMGCAAAA',134.34,100
+'1998-01-21','AAAAAAAANAFEAAAA',134.42,100
+'1998-01-22','AAAAAAAAKGABAAAA',141.77,100
+'1998-01-23','AAAAAAAAKJIBAAAA',131.47,100
+'1998-01-24','AAAAAAAAFABBAAAA',139.93,100
+'1998-01-25','AAAAAAAAHHNBAAAA',136.40,100
+'1998-01-26','AAAAAAAAPIACAAAA',131.45,100
+'1998-01-27','AAAAAAAACBPBAAAA',138.18,100
+'1998-01-27','AAAAAAAAOPDDAAAA',138.18,100
+'1998-01-28','AAAAAAAABHCAAAAA',130.44,100
+'1998-01-29','AAAAAAAAMELBAAAA',138.45,100
+'1998-01-30','AAAAAAAAIKLAAAAA',139.22,100
+'1998-01-31','AAAAAAAAOJADAAAA',133.87,100
+'1998-02-01','AAAAAAAAACHCAAAA',136.31,100
+'1998-02-02','AAAAAAAAKNCAAAAA',138.73,100
+'1998-02-03','AAAAAAAALDBBAAAA',139.99,100
+'1998-02-04','AAAAAAAAMIPAAAAA',141.97,100
+'1998-02-05','AAAAAAAAFPIDAAAA',127.45,100
+'1998-02-06','AAAAAAAAPNMAAAAA',136.83,100
+'1998-02-07','AAAAAAAAIDGAAAAA',131.90,100
+'1998-02-08','AAAAAAAAMNDAAAAA',129.21,100
+'1998-02-09','AAAAAAAAIAACAAAA',133.56,100
+'1998-02-10','AAAAAAAAAJHAAAAA',136.71,100
+'1998-02-11','AAAAAAAACHDCAAAA',136.11,100
+'1998-02-12','AAAAAAAAOAOBAAAA',137.46,100
+'1998-02-13','AAAAAAAAAPEAAAAA',136.12,100
+'1998-02-14','AAAAAAAAMDEBAAAA',140.04,100
+'1998-02-15','AAAAAAAAJBMAAAAA',129.20,100
+'1998-02-16','AAAAAAAAHECAAAAA',134.73,100
+'1998-02-17','AAAAAAAAEDGDAAAA',133.28,100
+'1998-02-18','AAAAAAAALMBEAAAA',136.95,100
+'1998-02-19','AAAAAAAANIPBAAAA',135.70,100
+'1998-02-20','AAAAAAAAALBCAAAA',139.14,100
+---- TYPES
+STRING, STRING, DECIMAL, BIGINT
+====
+---- QUERY
+# Partitioned Top-N with a high number of partitions and a larger limit returning many
+# columns. Memory pressure will force in-memory partitions to be evicted.
+select * from (
+  select d_date, s_store_name, s_state, i_item_id, ss_list_price,
+        ss_wholesale_cost, ss_sales_price, ss_ext_discount_amt, ss_ext_sales_price,
+        ss_ext_wholesale_cost, ss_ext_list_price, ss_ext_tax, ss_coupon_amt, ss_net_paid,
+        ss_net_paid_inc_tax, ss_net_profit,
+        rank() over (partition by ss_sold_date_sk order by ss_list_price desc) rnk
+  from store_sales ss
+      join item i on ss_item_sk = i_item_sk
+      join date_dim d on ss_sold_date_sk = d_date_sk
+      join store s on ss_store_sk = s_store_sk
+  where ss_list_price is not null) v
+where rnk = 500
+order by ss_list_price desc, d_date, s_store_name
+limit 50
+---- RESULTS
+'2000-01-02','able','TN','AAAAAAAAMGCAAAAA',133.50,77.17,88.11,0.00,7753.68,6790.96,11748.00,232.61,0.00,7753.68,7986.29,962.72,500
+'1999-01-02','ese','TN','AAAAAAAADCCDAAAA',132.62,94.06,125.98,0.00,11842.12,8841.64,12466.28,118.42,0.00,11842.12,11960.54,3000.48,500
+'2002-01-02','bar','TN','AAAAAAAAPPABAAAA',131.59,70.37,75.00,935.25,2175.00,2040.73,3816.11,111.57,935.25,1239.75,1351.32,-800.98,500
+'1998-12-19','bar','TN','AAAAAAAAMBNAAAAA',131.49,75.14,69.68,0.00,975.52,1051.96,1840.86,9.75,0.00,975.52,985.27,-76.44,500
+'2002-12-30','ought','TN','AAAAAAAAGLKCAAAA',130.46,69.77,63.92,0.00,5944.56,6488.61,12132.78,237.78,0.00,5944.56,6182.34,-544.05,500
+'1999-12-28','able','TN','AAAAAAAACGCEAAAA',129.78,78.66,88.25,757.18,3441.75,3067.74,5061.42,80.53,757.18,2684.57,2765.10,-383.17,500
+'2001-12-30','eing','TN','AAAAAAAACBHBAAAA',128.66,NULL,113.22,NULL,2151.18,NULL,2444.54,NULL,NULL,NULL,2258.73,815.29,500
+'2002-11-13','bar','TN','AAAAAAAACAFBAAAA',128.56,64.28,88.70,372.54,443.50,321.40,642.80,2.83,372.54,70.96,73.79,-250.44,500
+'1999-12-24','able','TN','AAAAAAAAAHJBAAAA',128.48,80.81,111.77,0.00,2347.17,1697.01,2698.08,117.35,0.00,2347.17,2464.52,650.16,500
+'2002-12-23','able','TN','AAAAAAAAKCGBAAAA',128.41,69.04,15.40,0.00,523.60,2347.36,4365.94,20.94,0.00,523.60,544.54,-1823.76,500
+'2002-12-19','ation','TN','AAAAAAAAEHFBAAAA',128.27,86.09,33.35,0.00,3101.55,8006.37,11929.11,186.09,0.00,3101.55,3287.64,-4904.82,500
+'2001-12-18','bar','TN','AAAAAAAAGOCAAAAA',128.16,72.82,26.91,0.00,322.92,873.84,1537.92,16.14,0.00,322.92,339.06,-550.92,500
+'2002-12-01','ese','TN','AAAAAAAAIPBBAAAA',127.91,72.27,23.02,0.00,575.50,1806.75,3197.75,0.00,0.00,575.50,575.50,-1231.25,500
+'2001-12-04','eing','TN','AAAAAAAAIGDDAAAA',127.64,96.70,91.90,3400.30,4595.00,4835.00,6382.00,71.68,3400.30,1194.70,1266.38,-3640.30,500
+'2001-12-04','ought','TN','AAAAAAAAFMIAAAAA',127.64,94.55,98.28,0.00,8353.80,8036.75,10849.40,250.61,0.00,8353.80,8604.41,317.05,500
+'1998-12-26','able','TN','AAAAAAAAGMABAAAA',127.55,89.20,70.15,0.00,3998.55,5084.40,7270.35,319.88,0.00,3998.55,4318.43,-1085.85,500
+'1998-11-17','eing','TN','AAAAAAAAKFNBAAAA',127.45,70.03,93.03,0.00,6046.95,4551.95,8284.25,544.22,0.00,6046.95,6591.17,1495.00,500
+'1998-11-20','ation','TN','AAAAAAAAHDFBAAAA',127.40,76.29,112.11,0.00,6950.82,4729.98,7898.80,486.55,0.00,6950.82,7437.37,2220.84,500
+'1998-11-27','eing','TN','AAAAAAAAPKDAAAAA',127.26,71.10,100.53,1483.82,8243.46,5830.20,10435.32,608.36,1483.82,6759.64,7368.00,929.44,500
+'2002-11-16','bar','TN','AAAAAAAAOHDBAAAA',127.22,91.53,43.25,216.25,1081.25,2288.25,3180.50,8.65,216.25,865.00,873.65,-1423.25,500
+'2000-11-04','able','TN','AAAAAAAAIKIDAAAA',127.21,80.01,22.89,0.00,114.45,400.05,636.05,0.00,0.00,114.45,114.45,-285.60,500
+'2000-12-02','eing','TN','AAAAAAAAEEDAAAAA',127.20,63.92,68.68,0.00,5494.40,5113.60,10176.00,219.77,0.00,5494.40,5714.17,380.80,500
+'1999-12-18','able','TN','AAAAAAAAIJBCAAAA',127.07,68.32,20.33,0.00,853.86,2869.44,5336.94,25.61,0.00,853.86,879.47,-2015.58,500
+'1998-12-23','ought','TN','AAAAAAAAELJBAAAA',126.99,79.87,125.72,0.00,2640.12,1677.27,2666.79,79.20,0.00,2640.12,2719.32,962.85,500
+'1998-12-04','able','TN','AAAAAAAAEPAAAAAA',126.97,NULL,NULL,NULL,NULL,586.02,761.82,NULL,NULL,617.04,660.23,NULL,500
+'2000-12-31','bar','TN','AAAAAAAAAANBAAAA',126.79,81.80,112.84,0.00,7221.76,5235.20,8114.56,433.30,0.00,7221.76,7655.06,1986.56,500
+'2000-12-17','ation','TN','AAAAAAAAGBLBAAAA',126.68,68.11,15.20,665.76,912.00,4086.60,7600.80,4.92,665.76,246.24,251.16,-3840.36,500
+'2000-12-09','eing','TN','AAAAAAAAHMAAAAAA',126.63,70.35,100.03,0.00,400.12,281.40,506.52,20.00,0.00,400.12,420.12,118.72,500
+'2001-12-09','able','TN','AAAAAAAAHGCEAAAA',126.59,86.12,46.83,758.64,936.60,1722.40,2531.80,3.55,758.64,177.96,181.51,-1544.44,500
+'1998-12-12','bar','TN','AAAAAAAAMNEAAAAA',126.58,91.73,45.56,0.00,2232.44,4494.77,6202.42,66.97,0.00,2232.44,2299.41,-2262.33,500
+'2001-12-24','ation','TN','AAAAAAAAOHKBAAAA',126.58,83.83,31.64,0.00,1455.44,3856.18,5822.68,29.10,0.00,1455.44,1484.54,-2400.74,500
+'2002-11-26','ation','TN','AAAAAAAACKFCAAAA',126.57,69.93,32.90,0.00,1677.90,3566.43,6455.07,16.77,0.00,1677.90,1694.67,-1888.53,500
+'1998-11-11','ation','TN','AAAAAAAAGOBCAAAA',126.56,69.54,93.65,0.00,5338.05,3963.78,7213.92,213.52,0.00,5338.05,5551.57,1374.27,500
+'2000-11-09','ought','TN','AAAAAAAACOCBAAAA',126.52,80.59,121.45,0.00,728.70,483.54,759.12,14.57,0.00,728.70,743.27,245.16,500
+'2001-11-20','ese','TN','AAAAAAAACOLCAAAA',126.43,98.01,89.76,0.00,2872.32,3136.32,4045.76,143.61,0.00,2872.32,3015.93,-264.00,500
+'1998-11-08','bar','TN','AAAAAAAAMNMDAAAA',126.38,89.00,37.91,0.00,2274.60,5340.00,7582.80,204.71,0.00,2274.60,2479.31,-3065.40,500
+'1999-11-14','able','TN','AAAAAAAABILCAAAA',126.37,90.27,11.37,386.23,898.23,7131.33,9983.23,5.12,386.23,512.00,517.12,-6619.33,500
+'2001-11-10','ation','TN','AAAAAAAAAFACAAAA',126.36,90.91,0.00,0.00,0.00,4909.14,6823.44,0.00,0.00,0.00,0.00,-4909.14,500
+'2001-11-16','able','TN','AAAAAAAAGHNDAAAA',126.36,78.49,48.01,0.00,3504.73,5729.77,9224.28,245.33,0.00,3504.73,3750.06,-2225.04,500
+'1998-12-06','eing','TN','AAAAAAAAIHFAAAAA',126.35,82.05,53.06,0.00,2546.88,3938.40,6064.80,0.00,0.00,2546.88,2546.88,-1391.52,500
+'2001-12-19','ese','TN','AAAAAAAAOLADAAAA',126.34,76.11,92.22,3750.58,7654.26,6317.13,10486.22,0.00,3750.58,3903.68,3903.68,-2413.45,500
+'1998-11-28','ought','TN','AAAAAAAAKAPDAAAA',126.28,88.31,90.92,0.00,6909.92,6711.56,9597.28,552.79,0.00,6909.92,7462.71,198.36,500
+'1999-11-30','ought','TN','AAAAAAAAKOJCAAAA',126.27,78.43,18.94,0.00,113.64,470.58,757.62,0.00,0.00,113.64,113.64,-356.94,500
+'2001-12-05','ese','TN','AAAAAAAAFMKBAAAA',126.20,74.68,80.76,0.00,7995.24,7393.32,12493.80,399.76,0.00,7995.24,8395.00,601.92,500
+'1999-11-07','bar','TN','AAAAAAAALOCBAAAA',126.17,97.06,109.76,0.00,2634.24,2329.44,3028.08,131.71,0.00,2634.24,2765.95,304.80,500
+'2000-11-10','eing','TN','AAAAAAAAIGGDAAAA',126.10,74.62,104.66,263.74,732.62,522.34,882.70,23.44,263.74,468.88,492.32,-53.46,500
+'1999-01-01','ought','TN','AAAAAAAAGNMDAAAA',126.08,96.25,108.42,0.00,1626.30,1443.75,1891.20,113.84,0.00,1626.30,1740.14,182.55,500
+'2000-11-19','able','TN','AAAAAAAAFEKAAAAA',126.05,70.82,99.57,0.00,4779.36,3399.36,6050.40,238.96,0.00,4779.36,5018.32,1380.00,500
+'1999-12-20','ation','TN','AAAAAAAALGIDAAAA',125.98,66.31,119.68,0.00,4428.16,2453.47,4661.26,265.68,0.00,4428.16,4693.84,1974.69,500
+'1998-11-04','ought','TN','AAAAAAAAKOGCAAAA',125.97,65.61,123.45,2755.40,7653.90,4067.82,7810.14,440.86,2755.40,4898.50,5339.36,830.68,500
+---- TYPES
+STRING, STRING, STRING, STRING, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, BIGINT
+---- RUNTIME_PROFILE
+# Confirm that heaps were actually evicted due to memory pressure.
+row_regex: .*InMemoryHeapsEvicted: .* \([1-9][0-9]*\)
+====
+---- QUERY
+# Same as above with in-memory heaps evicted, but calculate aggregates over all the result
+# columns to ensure that the right number of rows are returned for all partitions in
+# the partitioned top-n.
+select min(d_date), min(s_store_name), min(s_state), min(i_item_id), avg(ss_list_price),
+        avg(ss_wholesale_cost), avg(ss_sales_price), avg(ss_ext_discount_amt),
+        avg(ss_ext_sales_price), avg(ss_ext_wholesale_cost), avg(ss_ext_list_price),
+        avg(ss_ext_tax), avg(ss_coupon_amt), avg(ss_net_paid), avg(ss_net_paid_inc_tax),
+        avg(ss_net_profit)
+from (
+  select d_date, s_store_name, s_state, i_item_id, ss_list_price,
+        ss_wholesale_cost, ss_sales_price, ss_ext_discount_amt, ss_ext_sales_price,
+        ss_ext_wholesale_cost, ss_ext_list_price, ss_ext_tax, ss_coupon_amt, ss_net_paid,
+        ss_net_paid_inc_tax, ss_net_profit,
+        rank() over (partition by ss_sold_date_sk order by ss_list_price desc) rnk
+  from store_sales ss
+      join item i on ss_item_sk = i_item_sk
+      join date_dim d on ss_sold_date_sk = d_date_sk
+      join store s on ss_store_sk = s_store_sk
+  where ss_list_price is not null) v
+where rnk < 500
+---- RESULTS
+'1998-01-02','able','TN','AAAAAAAAAAABAAAA',120.822125,75.697178,60.469191,305.284017,3058.201402,3824.409840,6104.842416,123.872886,305.284017,2752.882153,2876.854635,-1071.689798
+---- TYPES
+STRING, STRING, STRING, STRING, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL, DECIMAL
+---- RUNTIME_PROFILE
+# Confirm that heaps were actually evicted due to memory pressure.
+row_regex: .*InMemoryHeapsEvicted: .* \([1-9][0-9]*\)
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/partitioned-top-n.test b/testdata/workloads/functional-query/queries/QueryTest/partitioned-top-n.test
new file mode 100644
index 0000000..2c29b73
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/partitioned-top-n.test
@@ -0,0 +1,82 @@
+====
+---- QUERY
+# In-memory partitioned top-N with some partitions that hit limit.
+select tinyint_col, id, rn from (
+  select *, row_number() over (partition by tinyint_col order by id) as rn
+  from alltypesagg where id % 777 = 0 or id % 10 = 7) v
+where rn <= 5
+order by tinyint_col, rn
+---- RESULTS
+1,2331,1
+2,4662,1
+3,6993,1
+4,1554,1
+4,9324,2
+5,3885,1
+6,6216,1
+7,7,1
+7,17,2
+7,27,3
+7,37,4
+7,47,5
+8,3108,1
+9,5439,1
+NULL,0,1
+NULL,0,2
+NULL,7770,3
+NULL,7770,4
+---- TYPES
+TINYINT, INT, BIGINT
+====
+---- QUERY
+# Same query with rank() predicate. There are ties in the NULL partition that
+# results in a different value.
+select tinyint_col, id, rnk from (
+  select *, rank() over (partition by tinyint_col order by id) as rnk
+  from alltypesagg where id % 777 = 0 or id % 10 = 7) v
+where rnk <= 5
+order by tinyint_col, rnk
+---- RESULTS
+1,2331,1
+2,4662,1
+3,6993,1
+4,1554,1
+4,9324,2
+5,3885,1
+6,6216,1
+7,7,1
+7,17,2
+7,27,3
+7,37,4
+7,47,5
+8,3108,1
+9,5439,1
+NULL,0,1
+NULL,0,1
+NULL,7770,3
+NULL,7770,3
+---- TYPES
+TINYINT, INT, BIGINT
+====
+----QUERY
+# Same query with lower limit. The tie in the NULL partition must be returned.
+select tinyint_col, id, rnk from (
+  select *, rank() over (partition by tinyint_col order by id) as rnk
+  from alltypesagg where id % 777 = 0 or id % 10 = 7) v
+where rnk <= 1
+order by tinyint_col, rnk
+---- RESULTS
+1,2331,1
+2,4662,1
+3,6993,1
+4,1554,1
+5,3885,1
+6,6216,1
+7,7,1
+8,3108,1
+9,5439,1
+NULL,0,1
+NULL,0,1
+---- TYPES
+TINYINT, INT, BIGINT
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling.test b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
index 592039e..3ef2efc 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
@@ -366,3 +366,69 @@ BIGINT
 # Verify that at least one of the joins was spilled.
 row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
 ====
+---- QUERY
+# Spilling partitioned top-N.
+# Find suppliers with two or more duplicate l_extendedprice values tied for last place.
+# This has a relatively high partition count because it partitions by suppliers, so
+# it ends up falling back to doing a full, spilling, sort.
+set buffer_pool_limit=100m;
+SELECT s_name, min(l_quantity), min(l_extendedprice), min(l_discount), min(l_tax),
+       count(*)
+FROM   (SELECT s_name, lineitem.*, Rank()
+                 OVER(PARTITION BY s_name
+                   ORDER BY l_extendedprice) AS rank
+        FROM   lineitem join supplier on l_suppkey = s_suppkey) a
+WHERE rank < 500
+group by s_name
+having count(*) > 500
+order by s_name
+---- RESULTS
+'Supplier#000001076',1.00,989.07,0.00,0.00,501
+'Supplier#000001645',1.00,1048.14,0.00,0.00,501
+'Supplier#000002145',1.00,1046.14,0.00,0.00,501
+'Supplier#000002197',1.00,1103.19,0.00,0.00,501
+'Supplier#000002269',1.00,1220.26,0.00,0.00,501
+'Supplier#000002435',1.00,1365.42,0.00,0.00,501
+'Supplier#000002644',1.00,1076.13,0.00,0.00,501
+'Supplier#000002741',1.00,1145.24,0.00,0.00,501
+'Supplier#000002827',1.00,1287.30,0.00,0.00,501
+'Supplier#000002986',1.00,1397.48,0.00,0.00,501
+'Supplier#000003304',1.00,1236.30,0.00,0.00,501
+'Supplier#000003631',1.00,1049.12,0.00,0.00,501
+'Supplier#000003646',1.00,1058.14,0.00,0.00,501
+'Supplier#000003695',1.00,1104.19,0.00,0.00,501
+'Supplier#000003811',1.00,1256.30,0.00,0.00,501
+'Supplier#000004068',1.00,981.06,0.00,0.00,501
+'Supplier#000004131',1.00,1054.13,0.00,0.00,501
+'Supplier#000004371',1.00,1284.37,0.00,0.00,501
+'Supplier#000004442',1.00,1395.44,0.00,0.00,501
+'Supplier#000004443',1.00,1346.44,0.00,0.00,501
+'Supplier#000004913',1.00,1368.39,0.00,0.00,501
+'Supplier#000004924',1.00,1330.42,0.00,0.00,501
+'Supplier#000004926',1.00,1426.41,0.00,0.00,501
+'Supplier#000004931',1.00,1368.42,0.00,0.00,501
+'Supplier#000004941',1.00,1368.43,0.00,0.00,501
+'Supplier#000005166',1.00,1080.16,0.00,0.00,501
+'Supplier#000005400',1.00,1299.39,0.00,0.00,501
+'Supplier#000005468',1.00,1367.46,0.00,0.00,501
+'Supplier#000005852',1.00,1254.35,0.00,0.00,501
+'Supplier#000006397',1.00,1342.39,0.00,0.00,501
+'Supplier#000007646',1.00,1045.14,0.00,0.00,501
+'Supplier#000007830',1.00,1234.32,0.00,0.00,501
+'Supplier#000008157',1.00,1144.15,0.00,0.00,501
+'Supplier#000008432',1.00,1414.41,0.00,0.00,501
+'Supplier#000008720',1.00,1143.21,0.00,0.00,501
+'Supplier#000008787',1.00,1192.28,0.00,0.00,501
+'Supplier#000008851',1.00,1251.35,0.00,0.00,501
+'Supplier#000008859',1.00,1259.35,0.00,0.00,501
+'Supplier#000009255',1.00,1163.25,0.00,0.00,501
+'Supplier#000009675',1.00,1139.14,0.00,0.00,501
+'Supplier#000009847',1.00,1262.34,0.00,0.00,501
+'Supplier#000009883',1.00,1298.37,0.00,0.00,501
+---- TYPES
+STRING,DECIMAL,DECIMAL,DECIMAL,DECIMAL,BIGINT
+---- RUNTIME_PROFILE
+# Verify that at least one of the Top-N operators spilled.
+row_regex: .*InMemoryHeapsEvicted: .* \([1-9][0-9]*\)
+#row_regex: .*SpilledRuns: .* \([1-9][0-9]*\)
+====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/top-n.test b/testdata/workloads/functional-query/queries/QueryTest/top-n.test
index 84afbbc..4c05a29 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/top-n.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/top-n.test
@@ -1392,3 +1392,92 @@ offset 9223372036854775807
 ---- TYPES
 STRING
 ====
+---- QUERY
+# Basic Top-N query expressed with row_number() analytic function.
+select tinyint_col, cnt from (
+  select tinyint_col, count(*) cnt,
+         row_number() over (order by tinyint_col) as rn
+  from alltypesagg group by 1) v
+where rn <= 5
+order by 1
+---- RESULTS
+1,1000
+2,1000
+3,1000
+4,1000
+5,1000
+---- TYPES
+TINYINT, BIGINT
+====
+---- QUERY
+# Basic Top-N query expressed with row_number() analytic function with
+# NULLS FIRST ordering
+select tinyint_col, cnt from (
+  select tinyint_col, count(*) cnt,
+         row_number() over (order by tinyint_col NULLS FIRST) as rn
+  from alltypesagg group by 1) v
+where rn <= 5
+order by 1
+---- RESULTS
+1,1000
+2,1000
+3,1000
+4,1000
+NULL,2000
+---- TYPES
+TINYINT, BIGINT
+====
+---- QUERY
+# Basic Top-N query expressed with row_number() analytic function with
+# DESC ordering
+select tinyint_col, cnt from (
+  select tinyint_col, count(*) cnt,
+         row_number() over (order by tinyint_col DESC) as rn
+  from alltypesagg group by 1) v
+where rn <= 5
+order by 1
+---- RESULTS
+6,1000
+7,1000
+8,1000
+9,1000
+NULL,2000
+---- TYPES
+TINYINT, BIGINT
+====
+---- QUERY
+# Basic Top-N query expressed with rank() analytic function demonstrating
+# that it returns tied values.
+select tinyint_col, rnk from (
+  select tinyint_col, rank() over (order by tinyint_col DESC) as rnk
+  from alltypestiny) v
+where rnk <= 5
+order by rnk
+---- RESULTS
+1,1
+1,1
+1,1
+1,1
+0,5
+0,5
+0,5
+0,5
+---- TYPES
+TINYINT, BIGINT
+====
+---- QUERY
+# row_number() does not return ties above limit.
+select tinyint_col, rn from (
+  select tinyint_col, row_number() over (order by tinyint_col DESC) as rn
+  from alltypestiny) v
+where rn <= 5
+order by rn
+---- RESULTS
+1,1
+1,2
+1,3
+1,4
+0,5
+---- TYPES
+TINYINT, BIGINT
+====
diff --git a/testdata/workloads/targeted-perf/queries/primitive_orderby_all.test b/testdata/workloads/targeted-perf/queries/primitive_orderby_all.test
index 76c9355..8941f94 100644
--- a/testdata/workloads/targeted-perf/queries/primitive_orderby_all.test
+++ b/testdata/workloads/targeted-perf/queries/primitive_orderby_all.test
@@ -26,7 +26,7 @@ FROM (
   FROM lineitem
   WHERE l_shipdate < '1992-05-09'
   ) a
-WHERE rank < 10;
+WHERE rank < 10 or rank > 99999999999
 ---- RESULTS
 1
 2
diff --git a/testdata/workloads/targeted-perf/queries/primitive_orderby_bigint.test b/testdata/workloads/targeted-perf/queries/primitive_orderby_bigint.test
index 9ad3034..1b4c9b5 100644
--- a/testdata/workloads/targeted-perf/queries/primitive_orderby_bigint.test
+++ b/testdata/workloads/targeted-perf/queries/primitive_orderby_bigint.test
@@ -9,7 +9,7 @@ FROM   (SELECT Rank()
                    ORDER BY  l_orderkey) AS rank
         FROM   lineitem
         WHERE  l_shipdate < '1992-05-09') a
-WHERE  rank < 10;
+WHERE rank < 10 or rank > 99999999999
 ---- RESULTS
 ---- TYPES
 ====
diff --git a/testdata/workloads/targeted-perf/queries/primitive_orderby_bigint_expression.test b/testdata/workloads/targeted-perf/queries/primitive_orderby_bigint_expression.test
index a6b302f..7e8499a 100644
--- a/testdata/workloads/targeted-perf/queries/primitive_orderby_bigint_expression.test
+++ b/testdata/workloads/targeted-perf/queries/primitive_orderby_bigint_expression.test
@@ -9,7 +9,7 @@ FROM   (SELECT Rank()
                    ORDER BY  cast(substr( concat(cast(l_orderkey as string), cast (l_shipdate as string)),1,13) as bigint)) AS rank
         FROM   lineitem
         WHERE  l_shipdate < '1992-05-09') a
-WHERE  rank < 10;
+WHERE rank < 10 or rank > 99999999999
 ---- RESULTS
 ---- TYPES
 ====
diff --git a/testdata/workloads/targeted-perf/queries/primitive_top-n_partitioned.test b/testdata/workloads/targeted-perf/queries/primitive_top-n_partitioned.test
new file mode 100644
index 0000000..f3aa8e6
--- /dev/null
+++ b/testdata/workloads/targeted-perf/queries/primitive_top-n_partitioned.test
@@ -0,0 +1,45 @@
+====
+---- QUERY: primitive_top-n_partitioned
+-- Description : Scan a fact table and evaluate a partitioned top-n query
+SELECT *
+FROM   (SELECT l_shipmode, Rank()
+                 OVER(PARTITION BY l_shipmode
+                   ORDER BY l_orderkey) AS rank
+        FROM   lineitem
+        WHERE  l_shipdate < '1992-05-09') a
+WHERE rank < 5
+---- RESULTS:  VERIFY_IS_EQUAL_SORTED
+'AIR',1
+'AIR',2
+'AIR',3
+'AIR',4
+'FOB',1
+'FOB',2
+'FOB',3
+'FOB',4
+'MAIL',1
+'MAIL',2
+'MAIL',3
+'MAIL',4
+'RAIL',1
+'RAIL',2
+'RAIL',3
+'RAIL',4
+'REG AIR',1
+'REG AIR',1
+'REG AIR',3
+'REG AIR',4
+'REG AIR',4
+'SHIP',1
+'SHIP',2
+'SHIP',3
+'SHIP',4
+'TRUCK',1
+'TRUCK',2
+'TRUCK',3
+'TRUCK',4
+'TRUCK',4
+'TRUCK',4
+---- TYPES
+STRING,BIGINT
+====
diff --git a/tests/experiments/test_targeted_perf.py b/tests/experiments/test_targeted_perf.py
index c53d41d..ceef280 100644
--- a/tests/experiments/test_targeted_perf.py
+++ b/tests/experiments/test_targeted_perf.py
@@ -44,3 +44,6 @@ class TestTargetedPerf(ImpalaTestSuite):
 
   def test_perf_cancel_union(self, vector):
     self.run_test_case('primitive_cancel_union', vector)
+
+  def test_perf_topn_partitioned(self, vector):
+    self.run_test_case('primitive_top-n_partitioned', vector)
diff --git a/tests/query_test/test_analytic_tpcds.py b/tests/query_test/test_analytic_tpcds.py
index 67224fe..57042e8 100644
--- a/tests/query_test/test_analytic_tpcds.py
+++ b/tests/query_test/test_analytic_tpcds.py
@@ -44,3 +44,8 @@ class TestAnalyticTpcds(ImpalaTestSuite):
   def test_analytic_functions_tpcds(self, vector):
     vector.get_value('exec_option')['batch_size'] = vector.get_value('batch_size')
     self.run_test_case('QueryTest/analytic-fns-tpcds', vector)
+
+  def test_partitioned_topn(self, vector):
+    """Targeted tests for the partitioned top-n operator."""
+    vector.get_value('exec_option')['batch_size'] = vector.get_value('batch_size')
+    self.run_test_case('QueryTest/analytic-fns-tpcds-partitioned-topn', vector)
diff --git a/tests/query_test/test_queries.py b/tests/query_test/test_queries.py
index 249ee20..b3469db 100644
--- a/tests/query_test/test_queries.py
+++ b/tests/query_test/test_queries.py
@@ -132,10 +132,15 @@ class TestQueries(ImpalaTestSuite):
     if vector.get_value('table_format').file_format == 'hbase':
       pytest.xfail(reason="IMPALA-283 - select count(*) produces inconsistent results")
     vector.get_value('exec_option')['disable_outermost_topn'] = 1
+    vector.get_value('exec_option')['analytic_rank_pushdown_threshold'] = 0
     self.run_test_case('QueryTest/sort', vector)
     # We can get the sort tests for free from the top-n file
     self.run_test_case('QueryTest/top-n', vector)
 
+  def test_partitioned_top_n(self, vector):
+    """Test partitioned Top-N operator."""
+    self.run_test_case('QueryTest/partitioned-top-n', vector)
+
   def test_inline_view(self, vector):
     if vector.get_value('table_format').file_format == 'hbase':
       pytest.xfail("jointbl does not have columns with unique values, "