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 2019/02/25 07:57:14 UTC

[impala] branch 2.x updated (23922dd -> 93fa95a)

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

tarmstrong pushed a change to branch 2.x
in repository https://gitbox.apache.org/repos/asf/impala.git.


    from 23922dd  IMPALA-7191: don't call srand() at random times
     new c77b648  Ignore "IMPALA-7200: Fix missing FILESYSTEM_PREFIX hitting local dataload"
     new 34e444c  IMPALA-7121: Clean up partitionIds_ from HdfsTable
     new 91d0b64  [DOCS] Update to the workaround for IMPALA-3316
     new ac42195  IMPALA-7207: make Coordinator::exec_state_ an atomic enum
     new 63617bc  IMPALA-7140 (part 5): support fetching file info for FS tables
     new cb4bbbe  IMPALA-6425: reduce MemPool max chunk size
     new 0091cd6  IMPALA-7210: global debug actions should be case insensitive
     new a29bec1  IMPALA-6816: minimise calls to GetMinSubscriberTopicVersion()
     new 1219d8f  IMPALA-6305: Allow column definitions in ALTER VIEW
     new e5a243e  IMPALA-7205: Respond to ReportExecStatus() RPC with CANCELLED if query execution has terminated
     new f823c46  IMPALA-7215: Implement a templatized CountingBarrier
     new 16276f4  IMPALA-4848: Add WIDTH_BUCKET() function
     new adc41d5  IMPALA-110 (part 2): Refactor PartitionedAggregationNode
     new 93fa95a  IMPALA-7185: low statestore custom cluster interval

The 14 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 be/src/catalog/catalog-server.cc                   |    2 +-
 be/src/codegen/gen_ir_descriptions.py              |   14 +-
 be/src/codegen/impala-ir.cc                        |    3 +-
 be/src/common/atomic.h                             |    3 +-
 be/src/exec/CMakeLists.txt                         |   10 +-
 be/src/exec/aggregation-node.cc                    |  132 ++
 ...cardinality-check-node.h => aggregation-node.h} |   43 +-
 be/src/exec/aggregator.cc                          |  609 ++++++
 be/src/exec/aggregator.h                           |  211 +++
 be/src/exec/exec-node.cc                           |    9 +-
 be/src/exec/exec-node.h                            |    9 +-
 ...gation-node-ir.cc => grouping-aggregator-ir.cc} |  100 +-
 be/src/exec/grouping-aggregator-partition.cc       |  218 +++
 be/src/exec/grouping-aggregator.cc                 | 1098 +++++++++++
 ...ed-aggregation-node.h => grouping-aggregator.h} |  316 ++--
 be/src/exec/hdfs-scan-node.cc                      |    2 +-
 .../non-grouping-aggregator-ir.cc}                 |   15 +-
 be/src/exec/non-grouping-aggregator.cc             |  174 ++
 be/src/exec/non-grouping-aggregator.h              |  111 ++
 be/src/exec/partitioned-aggregation-node.cc        | 1955 --------------------
 be/src/exec/streaming-aggregation-node.cc          |  153 ++
 be/src/exec/streaming-aggregation-node.h           |   85 +
 be/src/exprs/expr-test.cc                          |   65 +
 be/src/exprs/math-functions-ir.cc                  |  156 +-
 be/src/exprs/math-functions.h                      |   15 +-
 be/src/rpc/rpc-mgr-test.cc                         |    6 +-
 be/src/runtime/coordinator-backend-state.cc        |   50 +-
 be/src/runtime/coordinator-backend-state.h         |   10 +-
 be/src/runtime/coordinator.cc                      |   32 +-
 be/src/runtime/coordinator.h                       |   12 +-
 be/src/runtime/fragment-instance-state.h           |    3 +-
 be/src/runtime/mem-pool.h                          |    5 +-
 be/src/scheduling/admission-controller.cc          |    9 +-
 be/src/scheduling/scheduler.cc                     |    2 +-
 be/src/service/impala-server.cc                    |   11 +-
 be/src/statestore/statestore-subscriber.cc         |    7 +-
 be/src/statestore/statestore-subscriber.h          |    6 +-
 be/src/statestore/statestore.cc                    |   50 +-
 be/src/statestore/statestore.h                     |    8 +-
 be/src/util/counting-barrier.h                     |   75 +-
 be/src/util/debug-util.cc                          |   11 +-
 be/src/util/hdfs-bulk-ops.cc                       |    2 +-
 be/src/util/string-util.cc                         |    8 +
 be/src/util/string-util.h                          |    4 +
 bin/ignored_commits.json                           |    4 +-
 common/function-registry/impala_functions.py       |    3 +-
 common/thrift/StatestoreService.thrift             |    6 +
 docs/topics/impala_known_issues.xml                |   17 +-
 fe/src/main/cup/sql-parser.cup                     |    5 +-
 .../org/apache/impala/analysis/AlterViewStmt.java  |   14 +-
 .../impala/analysis/CreateOrAlterViewStmtBase.java |    6 +-
 .../org/apache/impala/analysis/CreateViewStmt.java |    5 +-
 .../main/java/org/apache/impala/analysis/Expr.java |    2 +-
 .../java/org/apache/impala/catalog/FeFsTable.java  |    5 -
 .../org/apache/impala/catalog/HdfsPartition.java   |    4 +-
 .../java/org/apache/impala/catalog/HdfsTable.java  |   80 +-
 .../impala/catalog/local/DirectMetaProvider.java   |   19 +
 .../impala/catalog/local/LocalFsPartition.java     |   90 +-
 .../apache/impala/catalog/local/LocalFsTable.java  |   25 +-
 .../apache/impala/catalog/local/MetaProvider.java  |    9 +
 .../apache/impala/planner/HdfsPartitionPruner.java |    6 +-
 .../org/apache/impala/analysis/AnalyzeDDLTest.java |   37 +
 .../apache/impala/analysis/AuthorizationTest.java  |   23 +
 .../org/apache/impala/analysis/ParserTest.java     |   15 +
 .../impala/catalog/local/LocalCatalogTest.java     |   18 +
 .../queries/QueryTest/views-ddl.test               |   54 +
 tests/authorization/test_grant_revoke.py           |    7 +-
 tests/common/custom_cluster_test_suite.py          |   12 +-
 tests/query_test/test_cancellation.py              |   20 +-
 tests/statestore/test_statestore.py                |   93 +
 70 files changed, 3914 insertions(+), 2484 deletions(-)
 create mode 100644 be/src/exec/aggregation-node.cc
 copy be/src/exec/{cardinality-check-node.h => aggregation-node.h} (57%)
 create mode 100644 be/src/exec/aggregator.cc
 create mode 100644 be/src/exec/aggregator.h
 rename be/src/exec/{partitioned-aggregation-node-ir.cc => grouping-aggregator-ir.cc} (73%)
 create mode 100644 be/src/exec/grouping-aggregator-partition.cc
 create mode 100644 be/src/exec/grouping-aggregator.cc
 rename be/src/exec/{partitioned-aggregation-node.h => grouping-aggregator.h} (71%)
 copy be/src/{exprs/compound-predicates-ir.cc => exec/non-grouping-aggregator-ir.cc} (72%)
 create mode 100644 be/src/exec/non-grouping-aggregator.cc
 create mode 100644 be/src/exec/non-grouping-aggregator.h
 delete mode 100644 be/src/exec/partitioned-aggregation-node.cc
 create mode 100644 be/src/exec/streaming-aggregation-node.cc
 create mode 100644 be/src/exec/streaming-aggregation-node.h


[impala] 03/14: [DOCS] Update to the workaround for IMPALA-3316

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 91d0b6445a073ffe823f1560fdf16a5de7220a27
Author: Alex Rodoni <ar...@cloudera.com>
AuthorDate: Fri Jun 22 13:54:31 2018 -0700

    [DOCS] Update to the workaround for IMPALA-3316
    
    Both the date and time strings can be stored and converted to TIMESTAMP
    
    Change-Id: If45da5d24dd3bc5f649d95b5bc104047420dbea1
    Reviewed-on: http://gerrit.cloudera.org:8080/10800
    Reviewed-by: Csaba Ringhofer <cs...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-on: http://gerrit.cloudera.org:8080/12565
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
---
 docs/topics/impala_known_issues.xml | 17 +++++++++++++----
 1 file changed, 13 insertions(+), 4 deletions(-)

diff --git a/docs/topics/impala_known_issues.xml b/docs/topics/impala_known_issues.xml
index 322cdb5..8910328 100644
--- a/docs/topics/impala_known_issues.xml
+++ b/docs/topics/impala_known_issues.xml
@@ -508,10 +508,19 @@ select * from tab_separated; -- 20 second delay before getting "Cancelled due to
         </p>
 
         <p>
-          <b>Workaround:</b> If the <codeph>TIMESTAMP</codeph> values stored in the table
-          represent dates only, with no time portion, consider storing them as strings in
-          <codeph>yyyy-MM-dd</codeph> format. Impala implicitly converts such string values to
-          <codeph>TIMESTAMP</codeph> in calls to date/time functions.
+          <b>Workaround:</b>Store the <codeph>TIMESTAMP</codeph> values as
+          strings in one of the following formats:
+          <ul>
+            <li><codeph>yyyy-MM-dd</codeph></li>
+            <li><codeph>yyyy-MM-dd HH:mm:ss</codeph></li>
+            <li><codeph>yyyy-MM-dd HH:mm:ss.SSSSSSSSS</codeph>
+              <p>The date can
+                have the 1-9 digits in the fractional part.
+              </p>
+            </li>
+          </ul>
+          Impala implicitly converts such string values to
+            <codeph>TIMESTAMP</codeph> in calls to date/time functions.
         </p>
 
       </conbody>


[impala] 06/14: IMPALA-6425: reduce MemPool max chunk size

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit cb4bbbeebecda071f414cd180740ed1d3c594fb0
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Thu Jun 21 15:38:18 2018 -0700

    IMPALA-6425: reduce MemPool max chunk size
    
    IMPALA-2742 capped chunk size doubling to 1MB with the idea that 1MB
    would be a good value to amortise malloc() overhead. As it turns out,
    1MB allocations are significantly more expensive than smaller
    allocations because they don't have a central freelist in TCMalloc,
    so the previous choice was misinformed.
    
    Note that this doesn't prevent larger allocations from being served from
    the MemPool, it just caps the automatic doubling behaviour.
    
    Perf:
    Ran single-node TPC-H and targeted perf with scale factor 60. Overall
    improvement in mean/geomean of 0-2% and no regressions outside of the
    usual variance.
    
    Change-Id: I58cc3031ce592892cb9efe9ab41f07d86468b08c
    Reviewed-on: http://gerrit.cloudera.org:8080/10789
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/runtime/mem-pool.h | 5 +++--
 1 file changed, 3 insertions(+), 2 deletions(-)

diff --git a/be/src/runtime/mem-pool.h b/be/src/runtime/mem-pool.h
index 2d12da8..03f05e2 100644
--- a/be/src/runtime/mem-pool.h
+++ b/be/src/runtime/mem-pool.h
@@ -178,8 +178,9 @@ class MemPool {
   static const int INITIAL_CHUNK_SIZE = 4 * 1024;
 
   /// The maximum size of chunk that should be allocated. Allocations larger than this
-  /// size will get their own individual chunk.
-  static const int MAX_CHUNK_SIZE = 1024 * 1024;
+  /// size will get their own individual chunk. Chosen to be small enough that it gets
+  /// a freelist in TCMalloc's central cache.
+  static const int MAX_CHUNK_SIZE = 512 * 1024;
 
   struct ChunkInfo {
     uint8_t* data; // Owned by the ChunkInfo.


[impala] 09/14: IMPALA-6305: Allow column definitions in ALTER VIEW

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 1219d8fb11358c36284e459e0b9fd37d26f44df9
Author: poojanilangekar <po...@cloudera.com>
AuthorDate: Thu Jun 14 14:49:05 2018 -0700

    IMPALA-6305: Allow column definitions in ALTER VIEW
    
    This change adds support to change column definitions in ALTER VIEW
    statements. This support only required minor changes in the parser
    and the AlterViewStmt constructor.
    
    Here's an example syntax:
        alter view foo (a, b comment 'helloworld') as
        select * from bar;
    
        describe foo;
        +------+--------+------------+
        | name | type   | comment    |
        +------+--------+------------+
        | a    | string |            |
        | b    | string | helloworld |
        +------+--------+------------+
    
    The following tests were modified:
    1. ParserTest - To check that the parser handles column definitions
       for alter view statements.
    2. AnalyzerDDLTest - To ensure that the analyzer supports the
       change column definitions parsed.
    3. TestDdlStatements - To verify the end-to-end functioning of
       ALTER VIEW statements with change column definitions.
    4. AuthorizationTest - To ensure that alter table commands with
       column definitions check permissions as expected.
    
    Change-Id: I6073444a814a24d97e80df15fcd39be2812f63fc
    Reviewed-on: http://gerrit.cloudera.org:8080/10720
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 fe/src/main/cup/sql-parser.cup                     |  5 +-
 .../org/apache/impala/analysis/AlterViewStmt.java  | 14 ++++--
 .../impala/analysis/CreateOrAlterViewStmtBase.java |  6 +--
 .../org/apache/impala/analysis/CreateViewStmt.java |  5 +-
 .../org/apache/impala/analysis/AnalyzeDDLTest.java | 37 +++++++++++++++
 .../apache/impala/analysis/AuthorizationTest.java  | 23 +++++++++
 .../org/apache/impala/analysis/ParserTest.java     | 15 ++++++
 .../queries/QueryTest/views-ddl.test               | 54 ++++++++++++++++++++++
 8 files changed, 148 insertions(+), 11 deletions(-)

diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index db69aae..c2ded20 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -1869,8 +1869,9 @@ view_column_def ::=
   ;
 
 alter_view_stmt ::=
-  KW_ALTER KW_VIEW table_name:table KW_AS query_stmt:view_def
-  {: RESULT = new AlterViewStmt(table, view_def); :}
+  KW_ALTER KW_VIEW table_name:table view_column_defs:col_defs KW_AS
+  query_stmt:view_def
+  {: RESULT = new AlterViewStmt(table, col_defs, view_def); :}
   | KW_ALTER KW_VIEW table_name:before_table KW_RENAME KW_TO table_name:new_table
   {: RESULT = new AlterTableOrViewRenameStmt(before_table, new_table, false); :}
   ;
diff --git a/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java b/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
index a6b6548..cad0169 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AlterViewStmt.java
@@ -17,22 +17,27 @@
 
 package org.apache.impala.analysis;
 
+import java.util.List;
+
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.catalog.FeTable;
 import org.apache.impala.catalog.FeView;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.RuntimeEnv;
+import org.apache.impala.thrift.TAccessEvent;
+import org.apache.impala.thrift.TCatalogObjectType;
 import org.apache.impala.service.BackendConfig;
 
+import com.google.common.base.Joiner;
 import com.google.common.base.Preconditions;
 
 /**
  * Represents an ALTER VIEW AS statement.
  */
 public class AlterViewStmt extends CreateOrAlterViewStmtBase {
-
-  public AlterViewStmt(TableName tableName, QueryStmt viewDefStmt) {
-    super(false, tableName, null, null, viewDefStmt);
+  public AlterViewStmt(
+      TableName tableName, List<ColumnDef> columnDefs, QueryStmt viewDefStmt) {
+    super(false, tableName, columnDefs, null, viewDefStmt);
   }
 
   @Override
@@ -51,6 +56,8 @@ public class AlterViewStmt extends CreateOrAlterViewStmtBase {
       throw new AnalysisException(String.format(
           "ALTER VIEW not allowed on a table: %s.%s", dbName_, getTbl()));
     }
+    analyzer.addAccessEvent(new TAccessEvent(dbName_ + "." + tableName_.getTbl(),
+        TCatalogObjectType.VIEW, Privilege.ALTER.toString()));
 
     createColumnAndViewDefs(analyzer);
     if (BackendConfig.INSTANCE.getComputeLineage() || RuntimeEnv.INSTANCE.isTestEnv()) {
@@ -66,6 +73,7 @@ public class AlterViewStmt extends CreateOrAlterViewStmtBase {
       sb.append(tableName_.getDb() + ".");
     }
     sb.append(tableName_.getTbl());
+    if (columnDefs_ != null) sb.append("(" + Joiner.on(", ").join(columnDefs_) + ")");
     sb.append(" AS " + viewDefStmt_.toSql());
     return sb.toString();
   }
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java b/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
index 4310871..6cdec98 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateOrAlterViewStmtBase.java
@@ -40,7 +40,7 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
 
   protected final boolean ifNotExists_;
   protected final TableName tableName_;
-  protected final ArrayList<ColumnDef> columnDefs_;
+  protected final List<ColumnDef> columnDefs_;
   protected final String comment_;
   protected final QueryStmt viewDefStmt_;
 
@@ -67,10 +67,10 @@ public abstract class CreateOrAlterViewStmtBase extends StatementBase {
 
   // Columns to use in the select list of the expanded SQL string and when registering
   // this view in the metastore. Set in analysis.
-  protected ArrayList<ColumnDef> finalColDefs_;
+  protected List<ColumnDef> finalColDefs_;
 
   public CreateOrAlterViewStmtBase(boolean ifNotExists, TableName tableName,
-      ArrayList<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
+      List<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
     Preconditions.checkNotNull(tableName);
     Preconditions.checkNotNull(viewDefStmt);
     this.ifNotExists_ = ifNotExists;
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
index 6e98fe5..1a17aa8 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateViewStmt.java
@@ -17,7 +17,7 @@
 
 package org.apache.impala.analysis;
 
-import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.impala.authorization.Privilege;
 import org.apache.impala.common.AnalysisException;
@@ -33,9 +33,8 @@ import com.google.common.base.Preconditions;
  * Represents a CREATE VIEW statement.
  */
 public class CreateViewStmt extends CreateOrAlterViewStmtBase {
-
   public CreateViewStmt(boolean ifNotExists, TableName tableName,
-      ArrayList<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
+      List<ColumnDef> columnDefs, String comment, QueryStmt viewDefStmt) {
     super(ifNotExists, tableName, columnDefs, comment, viewDefStmt);
   }
 
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
index 634fb87..550ab24 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -1087,11 +1087,30 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     // View-definition references a view.
     AnalyzesOk("alter view functional.alltypes_view as " +
         "select * from functional.alltypes_view");
+    // Change column definitions.
+    AnalyzesOk("alter view functional.alltypes_view (a, b) as " +
+        "select int_col, string_col from functional.alltypes");
+    // Change column definitions after renaming columns in select.
+    AnalyzesOk("alter view functional.alltypes_view (a, b) as " +
+        "select int_col x, string_col y from functional.alltypes");
 
     // View-definition resulting in Hive-style auto-generated column names.
     AnalyzesOk("alter view functional.alltypes_view as " +
         "select trim('abc'), 17 * 7");
 
+    // Altering a view on a view is ok (alltypes_view is a view on alltypes).
+    AnalyzesOk("alter view functional.alltypes_view (aaa, bbb) as " +
+        "select * from functional.complex_view");
+
+    // Altering a view with same column as existing one.
+    AnalyzesOk("alter view functional.complex_view (abc, xyz) as " +
+        "select year, month from functional.alltypes_view");
+
+    // Alter view with joins and aggregates.
+    AnalyzesOk("alter view functional.alltypes_view (cnt) as " +
+        "select count(distinct x.int_col) from functional.alltypessmall x " +
+        "inner join functional.alltypessmall y on (x.id = y.id) group by x.bigint_col");
+
     // Cannot ALTER VIEW a table.
     AnalysisError("alter view functional.alltypes as " +
         "select * from functional.alltypesagg",
@@ -1127,6 +1146,24 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalyzesOk("alter view functional.alltypes_view as " +
         "select * from functional.alltypestiny where id in " +
         "(select id from functional.alltypessmall where int_col = 1)");
+    // Mismatching number of columns in column definition and view-alteration statement.
+    AnalysisError("alter view functional.alltypes_view (a) as " +
+        "select int_col, string_col from functional.alltypes",
+        "Column-definition list has fewer columns (1) than the " +
+        "view-definition query statement returns (2).");
+    AnalysisError("alter view functional.alltypes_view (a, b, c) as " +
+        "select int_col from functional.alltypes",
+        "Column-definition list has more columns (3) than the " +
+        "view-definition query statement returns (1).");
+    // Duplicate columns in the view-alteration statement.
+    AnalysisError("alter view functional.alltypes_view as " +
+        "select * from functional.alltypessmall a " +
+        "inner join functional.alltypessmall b on a.id = b.id",
+        "Duplicate column name: id");
+    // Duplicate columns in the column definition.
+    AnalysisError("alter view functional.alltypes_view (a, b, a) as " +
+        "select int_col, int_col, int_col from functional.alltypes",
+        "Duplicate column name: a");
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
index 36bcaa3..a684847 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
@@ -1656,6 +1656,8 @@ public class AuthorizationTest extends FrontendTestBase {
   public void TestAlterView() throws ImpalaException {
     AuthzOk("ALTER VIEW functional_seq_snap.alltypes_view rename to " +
         "functional_seq_snap.v1");
+    AuthzOk("ALTER VIEW functional.alltypes_view (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg");
 
     // ALTER privilege on view only. RENAME also requires CREATE privileges on the DB.
     AuthzOk("ALTER VIEW functional.alltypes_view rename to functional_seq_snap.view_view_1");
@@ -1682,6 +1684,10 @@ public class AuthorizationTest extends FrontendTestBase {
         "functional_seq_snap.new_view",
         "User '%s' does not have privileges to execute 'ALTER' on: " +
         "functional.alltypes");
+    AuthzError("ALTER VIEW functional.alltypes (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: " +
+        "functional.alltypes");
 
     // Rename view that does not exist (no permissions).
     AuthzError("ALTER VIEW functional.notbl rename to functional_seq_snap.newtbl",
@@ -1694,26 +1700,43 @@ public class AuthorizationTest extends FrontendTestBase {
     // Alter view that does not exist (no permissions).
     AuthzError("ALTER VIEW functional.notbl rename to functional_seq_snap.new_view",
         "User '%s' does not have privileges to execute 'ALTER' on: functional.notbl");
+    AuthzError("ALTER VIEW functional.notbl (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: functional.notbl");
 
     // Alter view in db that does not exist (no permissions).
     AuthzError("ALTER VIEW nodb.alltypes rename to functional_seq_snap.new_view",
         "User '%s' does not have privileges to execute 'ALTER' on: nodb.alltypes");
+    AuthzError("ALTER VIEW nodb.alltypes (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: nodb.alltypes");
 
     // Unqualified view name.
     AuthzError("ALTER VIEW alltypes rename to functional_seq_snap.new_view",
         "User '%s' does not have privileges to execute 'ALTER' on: default.alltypes");
+    AuthzError("ALTER VIEW alltypes (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: default.alltypes");
 
     // No permissions on target view.
     AuthzError("alter view functional.alltypes_view_sub as " +
         "select * from functional.alltypesagg",
         "User '%s' does not have privileges to execute 'ALTER' on: " +
         "functional.alltypes_view");
+    AuthzError("alter view functional.alltypes_view_sub (a, b, c) as " +
+        "select int_col, string_col, timestamp_col from functional.alltypesagg",
+        "User '%s' does not have privileges to execute 'ALTER' on: " +
+        "functional.alltypes_view");
 
     // No permissions on source view.
     AuthzError("alter view functional_seq_snap.alltypes_view " +
         "as select * from functional.alltypes_view",
         "User '%s' does not have privileges to execute 'SELECT' on: " +
         "functional.alltypes_view");
+    AuthzError("alter view functional_seq_snap.alltypes_view (a, b, c) " +
+        "as select int_col, string_col, timestamp_col from functional.alltypes_view",
+        "User '%s' does not have privileges to execute 'SELECT' on: " +
+        "functional.alltypes_view");
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index 93c2e2b..1f19823 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -2906,12 +2906,19 @@ public class ParserTest extends FrontendTestBase {
     ParsesOk("ALTER VIEW Bar AS SELECT a, b, c FROM t");
     ParsesOk("ALTER VIEW Bar AS VALUES(1, 2, 3)");
     ParsesOk("ALTER VIEW Bar AS SELECT 1, 2, 3 UNION ALL select 4, 5, 6");
+    ParsesOk("ALTER VIEW Bar (x, y, z) AS SELECT a, b, c from t");
+    ParsesOk("ALTER VIEW Bar (x, y COMMENT 'foo', z) AS SELECT a, b, c from t");
 
     ParsesOk("ALTER VIEW Foo.Bar AS SELECT 1, 2, 3");
     ParsesOk("ALTER VIEW Foo.Bar AS SELECT a, b, c FROM t");
     ParsesOk("ALTER VIEW Foo.Bar AS VALUES(1, 2, 3)");
     ParsesOk("ALTER VIEW Foo.Bar AS SELECT 1, 2, 3 UNION ALL select 4, 5, 6");
     ParsesOk("ALTER VIEW Foo.Bar AS WITH t AS (SELECT 1, 2, 3) SELECT * FROM t");
+    ParsesOk("ALTER VIEW Foo.Bar (x, y, z) AS SELECT a, b, c from t");
+    ParsesOk("ALTER VIEW Foo.Bar (x, y, z COMMENT 'foo') AS SELECT a, b, c from t");
+
+    // Mismatched number of columns in column definition and view definition parses ok.
+    ParsesOk("ALTER VIEW Bar (x, y) AS SELECT 1, 2, 3");
 
     // Must be ALTER VIEW not ALTER TABLE.
     ParserError("ALTER TABLE Foo.Bar AS SELECT 1, 2, 3");
@@ -2921,6 +2928,14 @@ public class ParserTest extends FrontendTestBase {
     ParserError("ALTER VIEW Foo.Bar SELECT 1, 2, 3");
     // Missing view definition.
     ParserError("ALTER VIEW Foo.Bar AS");
+    // Empty column definition not allowed.
+    ParserError("ALTER VIEW Foo.Bar () AS SELECT c FROM t");
+    // Column definitions cannot include types.
+    ParserError("ALTER VIEW Foo.Bar (x int) AS SELECT c FROM t");
+    ParserError("ALTER VIEW Foo.Bar (x int COMMENT 'x') AS SELECT c FROM t");
+    // A type does not parse as an identifier.
+    ParserError("ALTER VIEW Foo.Bar (int COMMENT 'x') AS SELECT c FROM t");
+
     // Invalid view definitions. A view definition must be a query statement.
     ParserError("ALTER VIEW Foo.Bar AS INSERT INTO t select * from t");
     ParserError("ALTER VIEW Foo.Bar AS UPSERT INTO t select * from t");
diff --git a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
index 0461222..ae83e34 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/views-ddl.test
@@ -230,6 +230,60 @@ show tables in $DATABASE
 'view_on_view'
 ====
 ---- QUERY
+# Alter a view with predicates, joins, aggregates and order by
+alter view $DATABASE.complex_view (aaa comment 'abc', bbb comment 'xyz') as
+select count(a.bigint_col), b.string_col from
+functional.alltypesagg a inner join functional.alltypestiny b
+on a.id = b.id where a.bigint_col < 50
+group by b.string_col having count(a.bigint_col) > 1
+order by b.string_col limit 100
+---- RESULTS
+'View has been altered.'
+====
+---- QUERY
+# Test verifying the description of the altered view
+describe $DATABASE.complex_view
+---- RESULTS
+'aaa','bigint','abc'
+'bbb','string','xyz'
+---- TYPES
+string,string,string
+====
+---- QUERY
+# Test querying the altered view
+select * from $DATABASE.complex_view;
+---- RESULTS
+2,'0'
+2,'1'
+---- TYPES
+bigint,string
+====
+---- QUERY
+# Alter a view on a view
+alter view $DATABASE.view_on_view (foo, bar) as
+select * from $DATABASE.complex_view
+---- RESULTS
+'View has been altered.'
+====
+---- QUERY
+# Test describing the description of the altered view
+describe $DATABASE.view_on_view
+---- RESULTS
+'foo','bigint',''
+'bar','string',''
+---- TYPES
+string,string,string
+====
+---- QUERY
+# Test querying the altered view
+select foo, bar from $DATABASE.view_on_view;
+---- RESULTS
+2,'0'
+2,'1'
+---- TYPES
+bigint,string
+====
+---- QUERY
 # Test altering a with a new definition
 alter view $DATABASE.new_parquet_view as
 select bigint_col, string_col from functional_parquet.alltypesagg


[impala] 05/14: IMPALA-7140 (part 5): support fetching file info for FS tables

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 63617bc22bdc6190035e4ca0ccac8b5431c3dc30
Author: Todd Lipcon <to...@cloudera.com>
AuthorDate: Fri Jun 15 15:34:30 2018 -0700

    IMPALA-7140 (part 5): support fetching file info for FS tables
    
    This adds support for fetching file information and creating file
    descriptors.
    
    With this patch, I'm able to connect and run queries. Most planner tests
    still fail because of missing column stats resulting in different join
    orders compared to the existing implementation.
    
    Change-Id: I42d67ab754872fad094c7dacdd2e1182de1bf3e8
    Reviewed-on: http://gerrit.cloudera.org:8080/10749
    Reviewed-by: Vuk Ercegovac <ve...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
    Reviewed-on: http://gerrit.cloudera.org:8080/12569
    Reviewed-by: Quanlong Huang <hu...@gmail.com>
---
 .../java/org/apache/impala/catalog/FeFsTable.java  |  5 --
 .../org/apache/impala/catalog/HdfsPartition.java   |  4 +-
 .../java/org/apache/impala/catalog/HdfsTable.java  | 68 +++++++++++-----
 .../impala/catalog/local/DirectMetaProvider.java   | 19 +++++
 .../impala/catalog/local/LocalFsPartition.java     | 90 +++++++++++++++++++---
 .../apache/impala/catalog/local/LocalFsTable.java  | 25 +++---
 .../apache/impala/catalog/local/MetaProvider.java  |  9 +++
 .../impala/catalog/local/LocalCatalogTest.java     | 18 +++++
 8 files changed, 188 insertions(+), 50 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java b/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
index 508a0e5..73a7ffe 100644
--- a/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/FeFsTable.java
@@ -90,11 +90,6 @@ public interface FeFsTable extends FeTable {
   long getTotalHdfsBytes();
 
   /**
-   * @return the total number of files stored on HDFS for this table
-   */
-  long getTotalNumFiles();
-
-  /**
    * @return true if this table is backed by the Avro file format
    */
   boolean isAvroTable();
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsPartition.java b/fe/src/main/java/org/apache/impala/catalog/HdfsPartition.java
index 640aaa1..005f6f9 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsPartition.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsPartition.java
@@ -120,10 +120,8 @@ public class HdfsPartition implements FeFsPartition, PrunablePartition {
     /**
      * Creates the file descriptor of a file represented by 'fileStatus' that
      * resides in a filesystem that doesn't support the BlockLocation API (e.g. S3).
-     * 'fileFormat' is the file format of the partition where this file resides.
      */
-    public static FileDescriptor createWithNoBlocks(
-        FileStatus fileStatus, HdfsFileFormat fileFormat) {
+    public static FileDescriptor createWithNoBlocks(FileStatus fileStatus) {
       FlatBufferBuilder fbb = new FlatBufferBuilder(1);
       return new FileDescriptor(createFbFileDesc(fbb, fileStatus, null));
     }
diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index 34896cb..8ab33d8 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -284,7 +284,7 @@ public class HdfsTable extends Table implements FeFsTable {
       BackendConfig.INSTANCE.maxNonHdfsPartsParallelLoad();
 
   // File/Block metadata loading stats for a single HDFS path.
-  private class FileMetadataLoadStats {
+  public static class FileMetadataLoadStats {
     // Path corresponding to this metadata load request.
     private final Path hdfsPath;
 
@@ -409,10 +409,40 @@ public class HdfsTable extends Table implements FeFsTable {
     }
 
     FileSystem fs = partDir.getFileSystem(CONF);
-    boolean supportsBlocks = FileSystemUtil.supportsStorageIds(fs);
+
     RemoteIterator<LocatedFileStatus> fileStatusIter =
         FileSystemUtil.listFiles(fs, partDir, false);
     if (fileStatusIter == null) return loadStats;
+
+    List<FileDescriptor> newFileDescs = createFileDescriptors(
+        fs, fileStatusIter, hostIndex_, loadStats);
+    for (HdfsPartition partition: partitions) {
+      partition.setFileDescriptors(newFileDescs);
+    }
+    if (LOG.isTraceEnabled()) {
+      LOG.trace("Loaded file metadata for " + getFullName() + " " +
+          loadStats.debugString());
+    }
+    return loadStats;
+  }
+
+  /**
+   * Convert LocatedFileStatuses to FileDescriptors.
+   *
+   * If 'fs' is a FileSystem that supports block locations, the resulting
+   * descriptors include location information, and 'hostIndex' is updated
+   * to include all of the hosts referred to by the locations.
+   *
+   * 'loadStats' is updated to reflect this loading operation.
+   *
+   * May throw IOException if the provided RemoteIterator throws.
+   */
+  public static List<FileDescriptor> createFileDescriptors(
+      FileSystem fs,
+      RemoteIterator<LocatedFileStatus> fileStatusIter,
+      ListMap<TNetworkAddress> hostIndex,
+      FileMetadataLoadStats loadStats) throws IOException {
+    boolean supportsBlocks = FileSystemUtil.supportsStorageIds(fs);
     Reference<Long> numUnknownDiskIds = new Reference<Long>(Long.valueOf(0));
     List<FileDescriptor> newFileDescs = Lists.newArrayList();
     while (fileStatusIter.hasNext()) {
@@ -424,21 +454,15 @@ public class HdfsTable extends Table implements FeFsTable {
       FileDescriptor fd;
       if (supportsBlocks) {
         fd = FileDescriptor.create(fileStatus, fileStatus.getBlockLocations(), fs,
-            hostIndex_, numUnknownDiskIds);
+            hostIndex, numUnknownDiskIds);
       } else {
-        fd = FileDescriptor.createWithNoBlocks(
-            fileStatus, partitions.get(0).getFileFormat());
+        fd = FileDescriptor.createWithNoBlocks(fileStatus);
       }
       newFileDescs.add(fd);
       ++loadStats.loadedFiles;
     }
-    for (HdfsPartition partition: partitions) partition.setFileDescriptors(newFileDescs);
     loadStats.unknownDiskIds += numUnknownDiskIds.getRef();
-    if (LOG.isTraceEnabled()) {
-      LOG.trace("Loaded file metadata for " + getFullName() + " " +
-          loadStats.debugString());
-    }
-    return loadStats;
+    return newFileDescs;
   }
 
   /**
@@ -498,8 +522,7 @@ public class HdfsTable extends Table implements FeFsTable {
           fd = FileDescriptor.create(fileStatus, locations, fs, hostIndex_,
               numUnknownDiskIds);
         } else {
-          fd = FileDescriptor.createWithNoBlocks(
-              fileStatus, partitions.get(0).getFileFormat());
+          fd = FileDescriptor.createWithNoBlocks(fileStatus);
         }
         ++loadStats.loadedFiles;
       } else {
@@ -1755,9 +1778,6 @@ public class HdfsTable extends Table implements FeFsTable {
   public long getTotalHdfsBytes() { return fileMetadataStats_.totalFileBytes; }
 
   @Override // FeFsTable
-  public long getTotalNumFiles() { return fileMetadataStats_.numFiles; }
-
-  @Override // FeFsTable
   public String getHdfsBaseDir() { return hdfsBaseDir_; }
   public Path getHdfsBaseDirPath() { return new Path(hdfsBaseDir_); }
   @Override // FeFsTable
@@ -1987,7 +2007,14 @@ public class HdfsTable extends Table implements FeFsTable {
     Collections.sort(orderedPartitions, HdfsPartition.KV_COMPARATOR);
 
     long totalCachedBytes = 0L;
+    long totalBytes = 0L;
+    long totalNumFiles = 0L;
     for (FeFsPartition p: orderedPartitions) {
+      int numFiles = p.getFileDescriptors().size();
+      long size = p.getSize();
+      totalNumFiles += numFiles;
+      totalBytes += size;
+
       TResultRowBuilder rowBuilder = new TResultRowBuilder();
 
       // Add the partition-key values (as strings for simplicity).
@@ -2000,8 +2027,9 @@ public class HdfsTable extends Table implements FeFsTable {
       // Compute and report the extrapolated row count because the set of files could
       // have changed since we last computed stats for this partition. We also follow
       // this policy during scan-cardinality estimation.
-      if (statsExtrap) rowBuilder.add(table.getExtrapolatedNumRows(p.getSize()));
-      rowBuilder.add(p.getFileDescriptors().size()).addBytes(p.getSize());
+      if (statsExtrap) rowBuilder.add(table.getExtrapolatedNumRows(size));
+
+      rowBuilder.add(numFiles).addBytes(size);
       if (!p.isMarkedCached()) {
         // Helps to differentiate partitions that have 0B cached versus partitions
         // that are not marked as cached.
@@ -2055,8 +2083,8 @@ public class HdfsTable extends Table implements FeFsTable {
       if (statsExtrap) {
         rowBuilder.add(table.getExtrapolatedNumRows(table.getTotalHdfsBytes()));
       }
-      rowBuilder.add(table.getTotalNumFiles())
-          .addBytes(table.getTotalHdfsBytes())
+      rowBuilder.add(totalNumFiles)
+          .addBytes(totalBytes)
           .addBytes(totalCachedBytes).add("").add("").add("").add("");
       result.addToRows(rowBuilder.get());
     }
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java b/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
index e09f031..aadbf23 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/DirectMetaProvider.java
@@ -17,10 +17,16 @@
 
 package org.apache.impala.catalog.local;
 
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hive.common.FileUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
@@ -46,6 +52,7 @@ import com.google.common.collect.Maps;
  */
 class DirectMetaProvider implements MetaProvider {
   private static MetaStoreClientPool msClientPool_;
+  private static Configuration CONF = new Configuration();
 
   DirectMetaProvider() {
     initMsClientPool();
@@ -152,4 +159,16 @@ class DirectMetaProvider implements MetaProvider {
 
     return ret;
   }
+
+  @Override
+  public List<LocatedFileStatus> loadFileMetadata(Path dir) throws IOException {
+    Preconditions.checkNotNull(dir);
+    Preconditions.checkArgument(dir.isAbsolute(),
+        "Must pass absolute path: %s", dir);
+    FileSystem fs = dir.getFileSystem(CONF);
+    RemoteIterator<LocatedFileStatus> it = fs.listFiles(dir, /*recursive=*/false);
+    ImmutableList.Builder<LocatedFileStatus> b = new ImmutableList.Builder<>();
+    while (it.hasNext()) b.add(it.next());
+    return b.build();
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsPartition.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsPartition.java
index effe7c2..c5510d5 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsPartition.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsPartition.java
@@ -17,11 +17,17 @@
 
 package org.apache.impala.catalog.local;
 
-import java.util.Collections;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.impala.analysis.LiteralExpr;
 import org.apache.impala.catalog.FeCatalogUtils;
@@ -32,18 +38,22 @@ import org.apache.impala.catalog.HdfsPartition;
 import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
 import org.apache.impala.catalog.HdfsStorageDescriptor;
 import org.apache.impala.catalog.HdfsStorageDescriptor.InvalidStorageDescriptorException;
+import org.apache.impala.catalog.HdfsTable;
 import org.apache.impala.catalog.PartitionStatsUtil;
 import org.apache.impala.thrift.TAccessLevel;
 import org.apache.impala.thrift.THdfsPartitionLocation;
 import org.apache.impala.thrift.TPartitionStats;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Maps;
 
 public class LocalFsPartition implements FeFsPartition {
+  private static final Configuration CONF = new Configuration();
   private final LocalFsTable table_;
   private final LocalPartitionSpec spec_;
   private final Partition msPartition_;
+  private ImmutableList<FileDescriptor> fileDescriptors_;
 
   public LocalFsPartition(LocalFsTable table, LocalPartitionSpec spec,
       Partition msPartition) {
@@ -69,22 +79,20 @@ public class LocalFsPartition implements FeFsPartition {
 
   @Override
   public List<FileDescriptor> getFileDescriptors() {
-    // TODO(todd): implement me
-    return Collections.emptyList();
+    loadFileDescriptors();
+    return fileDescriptors_;
   }
 
   @Override
   public boolean hasFileDescriptors() {
-    // TODO(todd): implement file fetching. Return true for now
-    // so that partition pruning can be tested -- if we return false
-    // then all partitions would be pruned.
-    return true;
+    loadFileDescriptors();
+    return !fileDescriptors_.isEmpty();
   }
 
   @Override
   public int getNumFileDescriptors() {
-    // TODO Auto-generated method stub
-    return 0;
+    loadFileDescriptors();
+    return fileDescriptors_.size();
   }
 
   @Override
@@ -161,8 +169,12 @@ public class LocalFsPartition implements FeFsPartition {
 
   @Override
   public long getSize() {
-    // TODO Auto-generated method stub
-    return 0;
+    loadFileDescriptors();
+    long size = 0;
+    for (FileDescriptor fd : fileDescriptors_) {
+      size += fd.getFileLength();
+    }
+    return size;
   }
 
   @Override
@@ -203,4 +215,60 @@ public class LocalFsPartition implements FeFsPartition {
     return Maps.filterKeys(getParameters(),
         HdfsPartition.IS_NOT_INCREMENTAL_STATS_KEY);
   }
+
+
+  private void loadFileDescriptors() {
+    if (fileDescriptors_ != null) return;
+    Path partDir = getLocationPath();
+    List<LocatedFileStatus> stats;
+    try {
+      stats = table_.db_.getCatalog().getMetaProvider().loadFileMetadata(partDir);
+    } catch (FileNotFoundException fnf) {
+      // If the partition directory isn't found, this is treated as having no
+      // files.
+      fileDescriptors_ = ImmutableList.of();
+      return;
+    } catch (IOException ioe) {
+      throw new LocalCatalogException(String.format(
+          "Could not load files for partition %s of table %s",
+          spec_.getName(), table_.getFullName()), ioe);
+    }
+
+    HdfsTable.FileMetadataLoadStats loadStats =
+        new HdfsTable.FileMetadataLoadStats(partDir);
+
+    try {
+      FileSystem fs = partDir.getFileSystem(CONF);
+      fileDescriptors_ = ImmutableList.copyOf(
+          HdfsTable.createFileDescriptors(fs, new FakeRemoteIterator<>(stats),
+              table_.getHostIndex(), loadStats));
+    } catch (IOException e) {
+        throw new LocalCatalogException(String.format(
+            "Could not convert files to descriptors for partition %s of table %s",
+            spec_.getName(), table_.getFullName()), e);
+    }
+  }
+
+  /**
+   * Wrapper for a normal Iterable<T> to appear like a Hadoop RemoteIterator<T>.
+   * This is necessary because the existing code to convert file statuses to
+   * descriptors consumes the remote iterator directly and thus avoids materializing
+   * all of the LocatedFileStatus objects in memory at the same time.
+   */
+  private static class FakeRemoteIterator<T> implements RemoteIterator<T> {
+    private final Iterator<T> it_;
+
+    FakeRemoteIterator(Iterable<T> it) {
+      this.it_ = it.iterator();
+    }
+    @Override
+    public boolean hasNext() throws IOException {
+      return it_.hasNext();
+    }
+
+    @Override
+    public T next() throws IOException {
+      return it_.next();
+    }
+  }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
index f648ac8..6871f90 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/LocalFsTable.java
@@ -78,6 +78,12 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
    */
   private ArrayList<HashSet<Long>> nullPartitionIds_;
 
+  /**
+   * Map assigning integer indexes for the hosts containing blocks for this table.
+   * This is updated as a side effect of LocalFsPartition.loadFileDescriptors().
+   */
+  private final ListMap<TNetworkAddress> hostIndex_ = new ListMap<>();
+
   public LocalFsTable(LocalDb db, String tblName, SchemaInfo schemaInfo) {
     super(db, tblName, schemaInfo);
   }
@@ -113,7 +119,7 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
   @Override
   public TResultSet getFiles(List<List<TPartitionKeyValue>> partitionSet)
       throws CatalogException {
-    // TODO(todd): implement fetching files from HDFS
+    // TODO(todd): implement for SHOW FILES.
     return null;
   }
 
@@ -125,14 +131,12 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
 
   @Override
   public long getTotalHdfsBytes() {
-    // TODO Auto-generated method stub
-    return 0;
-  }
-
-  @Override
-  public long getTotalNumFiles() {
-    // TODO Auto-generated method stub
-    return 0;
+    // TODO(todd): this is slow because it requires loading all partitions. Remove if possible.
+    long size = 0;
+    for (FeFsPartition p: loadPartitions(getPartitionIds())) {
+      size += p.getSize();
+    }
+    return size;
   }
 
   @Override
@@ -387,7 +391,6 @@ public class LocalFsTable extends LocalTable implements FeFsTable {
 
   @Override
   public ListMap<TNetworkAddress> getHostIndex() {
-    // TODO(todd): implement me
-    return new ListMap<>();
+    return hostIndex_;
   }
 }
diff --git a/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java b/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
index 42bf40e..f59a7c9 100644
--- a/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
+++ b/fe/src/main/java/org/apache/impala/catalog/local/MetaProvider.java
@@ -17,9 +17,12 @@
 
 package org.apache.impala.catalog.local;
 
+import java.io.IOException;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.hadoop.fs.LocatedFileStatus;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
@@ -65,4 +68,10 @@ interface MetaProvider {
   Map<String, Partition> loadPartitionsByNames(String dbName, String tableName,
       List<String> partitionColumnNames, List<String> partitionNames)
       throws MetaException, TException;
+
+  /**
+   * Load file metadata and block locations for the files in the given
+   * partition directory.
+   */
+  List<LocatedFileStatus> loadFileMetadata(Path dir) throws IOException;
 }
diff --git a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
index aabb873..a242b1c 100644
--- a/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
+++ b/fe/src/test/java/org/apache/impala/catalog/local/LocalCatalogTest.java
@@ -19,6 +19,7 @@ package org.apache.impala.catalog.local;
 
 import static org.junit.Assert.*;
 
+import java.util.List;
 import java.util.Set;
 
 import org.apache.impala.catalog.CatalogTest;
@@ -27,6 +28,7 @@ import org.apache.impala.catalog.FeDb;
 import org.apache.impala.catalog.FeFsPartition;
 import org.apache.impala.catalog.FeFsTable;
 import org.apache.impala.catalog.FeTable;
+import org.apache.impala.catalog.HdfsPartition.FileDescriptor;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.thrift.TResultSet;
 import org.apache.impala.util.MetaStoreUtil;
@@ -131,4 +133,20 @@ public class LocalCatalogTest {
         t, Iterables.getOnlyElement(ids));
     assertTrue(partition.getPartitionValue(dayCol).isNullLiteral());
   }
+
+  @Test
+  public void testLoadFileDescriptors() throws Exception {
+    FeFsTable t = (FeFsTable) catalog_.getTable("functional",  "alltypes");
+    int totalFds = 0;
+    for (FeFsPartition p: FeCatalogUtils.loadAllPartitions(t)) {
+      List<FileDescriptor> fds = p.getFileDescriptors();
+      totalFds += fds.size();
+      for (FileDescriptor fd : fds) {
+        assertTrue(fd.getFileLength() > 0);
+        assertEquals(fd.getNumFileBlocks(), 1);
+        assertEquals(3, fd.getFbFileBlock(0).diskIdsLength());
+      }
+    }
+    assertEquals(24, totalFds);
+  }
 }


[impala] 10/14: IMPALA-7205: Respond to ReportExecStatus() RPC with CANCELLED if query execution has terminated

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e5a243e165baa92ae716f1535aaaf4fba5347aad
Author: Dan Hecht <dh...@cloudera.com>
AuthorDate: Mon Jun 25 12:05:16 2018 -0700

    IMPALA-7205: Respond to ReportExecStatus() RPC with CANCELLED if query execution has terminated
    
    Otherwise, if the coordinator to backend CancelFInstances() RPC had failed,
    the query can hang (and/or finstances can continue running until the
    query is closed.
    
    Testing:
    - the modified test reproduces the hang without the impalad fix
    
    Change-Id: I7bb2c26edace89853f14a329f891d1f9a065a991
    Reviewed-on: http://gerrit.cloudera.org:8080/10815
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/runtime/coordinator-backend-state.cc | 50 ++++++++++++++++-------------
 be/src/runtime/coordinator-backend-state.h  | 10 ++++--
 be/src/runtime/coordinator.cc               |  9 +++---
 tests/query_test/test_cancellation.py       | 20 ++++++++----
 4 files changed, 51 insertions(+), 38 deletions(-)

diff --git a/be/src/runtime/coordinator-backend-state.cc b/be/src/runtime/coordinator-backend-state.cc
index c871a48..a99acdb 100644
--- a/be/src/runtime/coordinator-backend-state.cc
+++ b/be/src/runtime/coordinator-backend-state.cc
@@ -40,8 +40,8 @@ using namespace rapidjson;
 namespace accumulators = boost::accumulators;
 
 Coordinator::BackendState::BackendState(
-    const TUniqueId& query_id, int state_idx, TRuntimeFilterMode::type filter_mode)
-  : query_id_(query_id),
+    const Coordinator& coord, int state_idx, TRuntimeFilterMode::type filter_mode)
+  : coord_(coord),
     state_idx_(state_idx),
     filter_mode_(filter_mode) {
 }
@@ -150,16 +150,16 @@ void Coordinator::BackendState::SetRpcParams(const DebugOptions& debug_options,
 }
 
 void Coordinator::BackendState::Exec(
-    const TQueryCtx& query_ctx, const DebugOptions& debug_options,
+    const DebugOptions& debug_options,
     const FilterRoutingTable& filter_routing_table,
     CountingBarrier* exec_complete_barrier) {
   NotifyBarrierOnExit notifier(exec_complete_barrier);
   TExecQueryFInstancesParams rpc_params;
-  rpc_params.__set_query_ctx(query_ctx);
+  rpc_params.__set_query_ctx(query_ctx());
   SetRpcParams(debug_options, filter_routing_table, &rpc_params);
   VLOG_FILE << "making rpc: ExecQueryFInstances"
       << " host=" << TNetworkAddressToString(impalad_address()) << " query_id="
-      << PrintId(query_id_);
+      << PrintId(query_id());
 
   // guard against concurrent UpdateBackendExecStatus() that may arrive after RPC returns
   lock_guard<mutex> l(lock_);
@@ -180,7 +180,7 @@ void Coordinator::BackendState::Exec(
 
   if (!rpc_status.ok()) {
     const string& err_msg =
-        Substitute(ERR_TEMPLATE, PrintId(query_id_), rpc_status.msg().msg());
+        Substitute(ERR_TEMPLATE, PrintId(query_id()), rpc_status.msg().msg());
     VLOG_QUERY << err_msg;
     status_ = Status::Expected(err_msg);
     return;
@@ -188,7 +188,7 @@ void Coordinator::BackendState::Exec(
 
   Status exec_status = Status(thrift_result.status);
   if (!exec_status.ok()) {
-    const string& err_msg = Substitute(ERR_TEMPLATE, PrintId(query_id_),
+    const string& err_msg = Substitute(ERR_TEMPLATE, PrintId(query_id()),
         exec_status.msg().GetFullMessageDetails());
     VLOG_QUERY << err_msg;
     status_ = Status::Expected(err_msg);
@@ -196,7 +196,7 @@ void Coordinator::BackendState::Exec(
   }
 
   for (const auto& entry: instance_stats_map_) entry.second->stopwatch_.Start();
-  VLOG_FILE << "rpc succeeded: ExecQueryFInstances query_id=" << PrintId(query_id_);
+  VLOG_FILE << "rpc succeeded: ExecQueryFInstances query_id=" << PrintId(query_id());
 }
 
 Status Coordinator::BackendState::GetStatus(bool* is_fragment_failure,
@@ -225,7 +225,7 @@ void Coordinator::BackendState::LogFirstInProgress(
   for (Coordinator::BackendState* backend_state : backend_states) {
     lock_guard<mutex> l(backend_state->lock_);
     if (!backend_state->IsDone()) {
-      VLOG_QUERY << "query_id=" << PrintId(backend_state->query_id_)
+      VLOG_QUERY << "query_id=" << PrintId(backend_state->query_id())
                  << ": first in-progress backend: "
                  << TNetworkAddressToString(backend_state->impalad_address());
       break;
@@ -351,9 +351,9 @@ bool Coordinator::BackendState::Cancel() {
 
   TCancelQueryFInstancesParams params;
   params.protocol_version = ImpalaInternalServiceVersion::V1;
-  params.__set_query_id(query_id_);
+  params.__set_query_id(query_id());
   TCancelQueryFInstancesResult dummy;
-  VLOG_QUERY << "sending CancelQueryFInstances rpc for query_id=" << PrintId(query_id_) <<
+  VLOG_QUERY << "sending CancelQueryFInstances rpc for query_id=" << PrintId(query_id()) <<
       " backend=" << TNetworkAddressToString(impalad_address());
 
   Status rpc_status;
@@ -362,26 +362,30 @@ bool Coordinator::BackendState::Cancel() {
   for (int i = 0; i < 3; ++i) {
     ImpalaBackendConnection backend_client(ExecEnv::GetInstance()->impalad_client_cache(),
         impalad_address(), &client_status);
-    if (client_status.ok()) {
-      // The return value 'dummy' is ignored as it's only set if the fragment instance
-      // cannot be found in the backend. The fragment instances of a query can all be
-      // cancelled locally in a backend due to RPC failure to coordinator. In which case,
-      // the query state can be gone already.
-      rpc_status = backend_client.DoRpc(
-          &ImpalaBackendClient::CancelQueryFInstances, params, &dummy);
-      if (rpc_status.ok()) break;
-    }
+    if (!client_status.ok()) continue;
+
+    rpc_status = DebugAction(query_ctx().client_request.query_options,
+        "COORD_CANCEL_QUERY_FINSTANCES_RPC");
+    if (!rpc_status.ok()) continue;
+
+    // The return value 'dummy' is ignored as it's only set if the fragment
+    // instance cannot be found in the backend. The fragment instances of a query
+    // can all be cancelled locally in a backend due to RPC failure to
+    // coordinator. In which case, the query state can be gone already.
+    rpc_status = backend_client.DoRpc(
+        &ImpalaBackendClient::CancelQueryFInstances, params, &dummy);
+    if (rpc_status.ok()) break;
   }
   if (!client_status.ok()) {
     status_.MergeStatus(client_status);
-    VLOG_QUERY << "CancelQueryFInstances query_id= " << PrintId(query_id_)
+    VLOG_QUERY << "CancelQueryFInstances query_id= " << PrintId(query_id())
                << " failed to connect to " << TNetworkAddressToString(impalad_address())
                << " :" << client_status.msg().msg();
     return true;
   }
   if (!rpc_status.ok()) {
     status_.MergeStatus(rpc_status);
-    VLOG_QUERY << "CancelQueryFInstances query_id= " << PrintId(query_id_)
+    VLOG_QUERY << "CancelQueryFInstances query_id= " << PrintId(query_id())
                << " rpc to " << TNetworkAddressToString(impalad_address())
                << " failed: " << rpc_status.msg().msg();
     return true;
@@ -390,7 +394,7 @@ bool Coordinator::BackendState::Cancel() {
 }
 
 void Coordinator::BackendState::PublishFilter(const TPublishFilterParams& rpc_params) {
-  DCHECK(rpc_params.dst_query_id == query_id_);
+  DCHECK(rpc_params.dst_query_id == query_id());
   {
     // If the backend is already done, it's not waiting for this filter, so we skip
     // sending it in this case.
diff --git a/be/src/runtime/coordinator-backend-state.h b/be/src/runtime/coordinator-backend-state.h
index e7af2e2..c51c16c 100644
--- a/be/src/runtime/coordinator-backend-state.h
+++ b/be/src/runtime/coordinator-backend-state.h
@@ -54,7 +54,7 @@ struct FInstanceExecParams;
 /// Thread-safe unless pointed out otherwise.
 class Coordinator::BackendState {
  public:
-  BackendState(const TUniqueId& query_id, int state_idx,
+  BackendState(const Coordinator& coord, int state_idx,
       TRuntimeFilterMode::type filter_mode);
 
   /// Creates InstanceStats for all instance in backend_exec_params in obj_pool
@@ -70,7 +70,7 @@ class Coordinator::BackendState {
   /// that weren't selected during its construction.
   /// The debug_options are applied to the appropriate TPlanFragmentInstanceCtxs, based
   /// on their node_id/instance_idx.
-  void Exec(const TQueryCtx& query_ctx, const DebugOptions& debug_options,
+  void Exec(const DebugOptions& debug_options,
       const FilterRoutingTable& filter_routing_table,
       CountingBarrier* rpc_complete_barrier);
 
@@ -202,7 +202,8 @@ class Coordinator::BackendState {
     void InitCounters();
   };
 
-  const TUniqueId query_id_;
+  const Coordinator& coord_; /// Coordinator object that owns this BackendState
+
   const int state_idx_;  /// index of 'this' in Coordinator::backend_states_
   const TRuntimeFilterMode::type filter_mode_;
 
@@ -256,6 +257,9 @@ class Coordinator::BackendState {
   /// Set in ApplyExecStatusReport(). Uses MonotonicMillis().
   int64_t last_report_time_ms_ = 0;
 
+  const TQueryCtx& query_ctx() const { return coord_.query_ctx(); }
+  const TUniqueId& query_id() const { return coord_.query_id(); }
+
   /// Fill in rpc_params based on state. Uses filter_routing_table to remove filters
   /// that weren't selected during its construction.
   void SetRpcParams(const DebugOptions& debug_options,
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 3489312..a240e85 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -205,7 +205,7 @@ void Coordinator::InitBackendStates() {
   int backend_idx = 0;
   for (const auto& entry: schedule_.per_backend_exec_params()) {
     BackendState* backend_state = obj_pool()->Add(
-        new BackendState(query_id(), backend_idx, filter_mode_));
+        new BackendState(*this, backend_idx, filter_mode_));
     backend_state->Init(entry.second, fragment_stats_, obj_pool());
     backend_states_[backend_idx++] = backend_state;
   }
@@ -335,7 +335,7 @@ void Coordinator::StartBackendExec() {
     ExecEnv::GetInstance()->exec_rpc_thread_pool()->Offer(
         [backend_state, this, &debug_options]() {
           DebugActionNoFail(schedule_.query_options(), "COORD_BEFORE_EXEC_RPC");
-          backend_state->Exec(query_ctx(), debug_options, filter_routing_table_,
+          backend_state->Exec(debug_options, filter_routing_table_,
               exec_rpcs_complete_barrier_.get());
         });
   }
@@ -708,10 +708,9 @@ Status Coordinator::UpdateBackendExecStatus(const TReportExecStatusParams& param
     // We've applied all changes from the final status report - notify waiting threads.
     backend_exec_complete_barrier_->Notify();
   }
-  // If all results have been returned, return a cancelled status to force the fragment
+  // If query execution has terminated, return a cancelled status to force the fragment
   // instance to stop executing.
-  // TODO: Make returning CANCELLED unnecessary with IMPALA-6984.
-  return ReturnedAllResults() ? Status::CANCELLED : Status::OK();
+  return exec_state_.Load() == ExecState::EXECUTING ? Status::OK() : Status::CANCELLED;
 }
 
 // TODO: add histogram/percentile
diff --git a/tests/query_test/test_cancellation.py b/tests/query_test/test_cancellation.py
index 4bcbff0..31d51e2 100644
--- a/tests/query_test/test_cancellation.py
+++ b/tests/query_test/test_cancellation.py
@@ -47,8 +47,11 @@ CANCEL_DELAY_IN_SECONDS = [0, 0.01, 0.1, 1, 4]
 # Number of times to execute/cancel each query under test
 NUM_CANCELATION_ITERATIONS = 1
 
-# Test cancellation on both running and hung queries
-DEBUG_ACTIONS = [None, 'WAIT']
+# Test cancellation on both running and hung queries. Node ID 0 is the scan node
+WAIT_ACTIONS = [None, '0:GETNEXT:WAIT']
+
+# Verify that failed CancelFInstances() RPCs don't lead to hung queries
+FAIL_RPC_ACTIONS = [None, 'COORD_CANCEL_QUERY_FINSTANCES_RPC:FAIL']
 
 # Verify close rpc running concurrently with fetch rpc. The two cases verify:
 # False: close and fetch rpc run concurrently.
@@ -75,7 +78,9 @@ class TestCancellation(ImpalaTestSuite):
     cls.ImpalaTestMatrix.add_dimension(
         ImpalaTestDimension('cancel_delay', *CANCEL_DELAY_IN_SECONDS))
     cls.ImpalaTestMatrix.add_dimension(
-        ImpalaTestDimension('action', *DEBUG_ACTIONS))
+        ImpalaTestDimension('wait_action', *WAIT_ACTIONS))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('fail_rpc_action', *FAIL_RPC_ACTIONS))
     cls.ImpalaTestMatrix.add_dimension(
         ImpalaTestDimension('join_before_close', *JOIN_BEFORE_CLOSE))
     cls.ImpalaTestMatrix.add_dimension(
@@ -125,9 +130,10 @@ class TestCancellation(ImpalaTestSuite):
             (file_format, query)
 
     join_before_close = vector.get_value('join_before_close')
-    action = vector.get_value('action')
-    # node ID 0 is the scan node
-    debug_action = '0:GETNEXT:' + action if action != None else ''
+    wait_action = vector.get_value('wait_action')
+    fail_rpc_action = vector.get_value('fail_rpc_action')
+
+    debug_action = "|".join(filter(None, [wait_action, fail_rpc_action]))
     vector.get_value('exec_option')['debug_action'] = debug_action
 
     vector.get_value('exec_option')['buffer_pool_limit'] =\
@@ -194,7 +200,7 @@ class TestCancellation(ImpalaTestSuite):
 
     # Executing the same query without canceling should work fine. Only do this if the
     # query has a limit or aggregation
-    if action is None and ('count' in query or 'limit' in query):
+    if not debug_action and ('count' in query or 'limit' in query):
       self.execute_query(query, vector.get_value('exec_option'))
 
   def teardown_method(self, method):


[impala] 13/14: IMPALA-110 (part 2): Refactor PartitionedAggregationNode

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit adc41d53c3e854839f8c77eaee875020824c6a5a
Author: Thomas Tauber-Marshall <tm...@cloudera.com>
AuthorDate: Thu May 10 19:39:57 2018 +0000

    IMPALA-110 (part 2): Refactor PartitionedAggregationNode
    
    This patch refactors PartitionedAggregationNode in preparation for
    supporting multiple distinct operators in a query.
    
    The primary goal of the refactor is to separate out the core
    aggregation functionality into a new type of object called an
    Aggregator. For now, each aggregation ExecNode will contain a single
    Aggregator. Then, future patches will extend the aggregation ExecNode
    to support taking a single input and processing it with multiple
    Aggregators, allowing us to support more exotic combinations of
    aggregate functions and groupings.
    
    Specifically, this patch splits PartitionedAggregationNode into five
    new classes:
    - Aggregator: a superclass containing the functionality that's shared
      between GroupingAggregator and NonGroupingAggregator.
    - GroupingAggregator: this class contains the bulk of the interesting
      aggregation code, including everything related to creating and
      updating partitions and hash tables, spilling, etc.
    - NonGroupingAggregator: this class handles the case of aggregations
      that don't have grouping exprs. Since these aggregations always
      result in just a single output row, the functionality here is
      relatively simple (eg. no spilling or streaming).
    - StreamingAggregationNode: this node performs a streaming
      preaggregation, where the input is retrieved from the child during
      GetNext() and passed to the GroupingAggregator (non-grouping do not
      support streaming) Eventually, we'll support a list of
      GroupingAggregators.
    - AggregationNode: this node performs a final aggregation, where the
      input is retrieved from the child during Open() and passed to the
      Aggregator. Currently the Aggregator can be either grouping or
      non-grouping. Eventually we'll support a list of GroupingAggregator
      and/or a single NonGroupingAggregator.
    
    Testing:
    - Passed a full exhaustive run.
    
    Change-Id: I9e7bb583f54aa4add3738bde7f57cf3511ac567e
    Reviewed-on: http://gerrit.cloudera.org:8080/10394
    Reviewed-by: Thomas Marshall <th...@cmu.edu>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/codegen/gen_ir_descriptions.py              |   14 +-
 be/src/codegen/impala-ir.cc                        |    3 +-
 be/src/exec/CMakeLists.txt                         |   10 +-
 be/src/exec/aggregation-node.cc                    |  132 ++
 be/src/exec/aggregation-node.h                     |   60 +
 be/src/exec/aggregator.cc                          |  609 ++++++
 be/src/exec/aggregator.h                           |  211 +++
 be/src/exec/exec-node.cc                           |    9 +-
 be/src/exec/exec-node.h                            |    9 +-
 ...gation-node-ir.cc => grouping-aggregator-ir.cc} |  100 +-
 be/src/exec/grouping-aggregator-partition.cc       |  218 +++
 be/src/exec/grouping-aggregator.cc                 | 1098 +++++++++++
 ...ed-aggregation-node.h => grouping-aggregator.h} |  316 ++--
 be/src/exec/non-grouping-aggregator-ir.cc          |   30 +
 be/src/exec/non-grouping-aggregator.cc             |  174 ++
 be/src/exec/non-grouping-aggregator.h              |  111 ++
 be/src/exec/partitioned-aggregation-node.cc        | 1955 --------------------
 be/src/exec/streaming-aggregation-node.cc          |  153 ++
 be/src/exec/streaming-aggregation-node.h           |   85 +
 19 files changed, 3056 insertions(+), 2241 deletions(-)

diff --git a/be/src/codegen/gen_ir_descriptions.py b/be/src/codegen/gen_ir_descriptions.py
index dd2df9e..99a97ae 100755
--- a/be/src/codegen/gen_ir_descriptions.py
+++ b/be/src/codegen/gen_ir_descriptions.py
@@ -47,14 +47,12 @@ ir_functions = [
    "_ZNK6impala14AggFnEvaluator11input_evalsEv"],
   ["AGG_FN_EVALUATOR_AGG_FN_CTX",
    "_ZNK6impala14AggFnEvaluator10agg_fn_ctxEv"],
-  ["PART_AGG_NODE_PROCESS_BATCH_UNAGGREGATED",
-   "_ZN6impala26PartitionedAggregationNode12ProcessBatchILb0EEENS_6StatusEPNS_8RowBatchENS_13TPrefetchMode4typeEPNS_12HashTableCtxE"],
-  ["PART_AGG_NODE_PROCESS_BATCH_AGGREGATED",
-   "_ZN6impala26PartitionedAggregationNode12ProcessBatchILb1EEENS_6StatusEPNS_8RowBatchENS_13TPrefetchMode4typeEPNS_12HashTableCtxE"],
-  ["PART_AGG_NODE_PROCESS_BATCH_NO_GROUPING",
-   "_ZN6impala26PartitionedAggregationNode22ProcessBatchNoGroupingEPNS_8RowBatchE"],
-  ["PART_AGG_NODE_PROCESS_BATCH_STREAMING",
-   "_ZN6impala26PartitionedAggregationNode21ProcessBatchStreamingEbNS_13TPrefetchMode4typeEPNS_8RowBatchES4_PNS_12HashTableCtxEPi"],
+  ["GROUPING_AGG_ADD_BATCH_IMPL",
+   "_ZN6impala18GroupingAggregator12AddBatchImplILb0EEENS_6StatusEPNS_8RowBatchENS_13TPrefetchMode4typeEPNS_12HashTableCtxE"],
+  ["NON_GROUPING_AGG_ADD_BATCH_IMPL",
+   "_ZN6impala21NonGroupingAggregator12AddBatchImplEPNS_8RowBatchE"],
+  ["GROUPING_AGG_ADD_BATCH_STREAMING_IMPL",
+   "_ZN6impala18GroupingAggregator21AddBatchStreamingImplEbNS_13TPrefetchMode4typeEPNS_8RowBatchES4_PNS_12HashTableCtxEPi"],
   ["AVG_UPDATE_BIGINT",
    "_ZN6impala18AggregateFunctions9AvgUpdateIN10impala_udf9BigIntValEEEvPNS2_15FunctionContextERKT_PNS2_9StringValE"],
   ["AVG_UPDATE_DOUBLE",
diff --git a/be/src/codegen/impala-ir.cc b/be/src/codegen/impala-ir.cc
index 9c5b3eb..0fa4fe9 100644
--- a/be/src/codegen/impala-ir.cc
+++ b/be/src/codegen/impala-ir.cc
@@ -26,11 +26,12 @@
 #pragma clang diagnostic ignored "-Wheader-hygiene"
 
 #include "codegen/codegen-anyval-ir.cc"
+#include "exec/grouping-aggregator-ir.cc"
 #include "exec/hash-table-ir.cc"
 #include "exec/hdfs-avro-scanner-ir.cc"
 #include "exec/hdfs-parquet-scanner-ir.cc"
 #include "exec/hdfs-scanner-ir.cc"
-#include "exec/partitioned-aggregation-node-ir.cc"
+#include "exec/non-grouping-aggregator-ir.cc"
 #include "exec/partitioned-hash-join-builder-ir.cc"
 #include "exec/partitioned-hash-join-node-ir.cc"
 #include "exec/select-node-ir.cc"
diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index 77c6e15..1753cb0 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -25,6 +25,8 @@ set(LIBRARY_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/exec")
 set(EXECUTABLE_OUTPUT_PATH "${BUILD_OUTPUT_ROOT_DIRECTORY}/exec")
 
 add_library(Exec
+  aggregation-node.cc
+  aggregator.cc
   analytic-eval-node.cc
   base-sequence-scanner.cc
   blocking-join-node.cc
@@ -38,6 +40,9 @@ add_library(Exec
   exchange-node.cc
   external-data-source-executor.cc
   filter-context.cc
+  grouping-aggregator.cc
+  grouping-aggregator-ir.cc
+  grouping-aggregator-partition.cc
   hash-table.cc
   hbase-table-sink.cc
   hbase-table-writer.cc
@@ -66,12 +71,12 @@ add_library(Exec
   incr-stats-util.cc
   nested-loop-join-builder.cc
   nested-loop-join-node.cc
+  non-grouping-aggregator.cc
+  non-grouping-aggregator-ir.cc
   parquet-column-readers.cc
   parquet-column-stats.cc
   parquet-metadata-utils.cc
   partial-sort-node.cc
-  partitioned-aggregation-node.cc
-  partitioned-aggregation-node-ir.cc
   partitioned-hash-join-builder.cc
   partitioned-hash-join-builder-ir.cc
   partitioned-hash-join-node.cc
@@ -91,6 +96,7 @@ add_library(Exec
   select-node-ir.cc
   singular-row-src-node.cc
   sort-node.cc
+  streaming-aggregation-node.cc
   subplan-node.cc
   text-converter.cc
   topn-node.cc
diff --git a/be/src/exec/aggregation-node.cc b/be/src/exec/aggregation-node.cc
new file mode 100644
index 0000000..d25284d
--- /dev/null
+++ b/be/src/exec/aggregation-node.cc
@@ -0,0 +1,132 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/aggregation-node.h"
+
+#include <sstream>
+
+#include "exec/grouping-aggregator.h"
+#include "exec/non-grouping-aggregator.h"
+#include "gutil/strings/substitute.h"
+#include "runtime/row-batch.h"
+#include "runtime/runtime-state.h"
+#include "util/debug-util.h"
+#include "util/runtime-profile-counters.h"
+
+#include "gen-cpp/PlanNodes_types.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+AggregationNode::AggregationNode(
+    ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
+  : ExecNode(pool, tnode, descs) {}
+
+Status AggregationNode::Init(const TPlanNode& tnode, RuntimeState* state) {
+  RETURN_IF_ERROR(ExecNode::Init(tnode, state));
+  if (tnode.agg_node.grouping_exprs.empty()) {
+    aggregator_.reset(new NonGroupingAggregator(this, pool_, tnode, state->desc_tbl()));
+  } else {
+    aggregator_.reset(new GroupingAggregator(this, pool_, tnode, state->desc_tbl()));
+  }
+  RETURN_IF_ERROR(aggregator_->Init(tnode, state));
+  runtime_profile_->AddChild(aggregator_->runtime_profile());
+  return Status::OK();
+}
+
+Status AggregationNode::Prepare(RuntimeState* state) {
+  SCOPED_TIMER(runtime_profile_->total_time_counter());
+  RETURN_IF_ERROR(ExecNode::Prepare(state));
+  aggregator_->SetDebugOptions(debug_options_);
+  RETURN_IF_ERROR(aggregator_->Prepare(state));
+  state->CheckAndAddCodegenDisabledMessage(runtime_profile());
+  return Status::OK();
+}
+
+void AggregationNode::Codegen(RuntimeState* state) {
+  DCHECK(state->ShouldCodegen());
+  ExecNode::Codegen(state);
+  if (IsNodeCodegenDisabled()) return;
+  aggregator_->Codegen(state);
+}
+
+Status AggregationNode::Open(RuntimeState* state) {
+  SCOPED_TIMER(runtime_profile_->total_time_counter());
+  // Open the child before consuming resources in this node.
+  RETURN_IF_ERROR(child(0)->Open(state));
+  RETURN_IF_ERROR(ExecNode::Open(state));
+
+  RETURN_IF_ERROR(aggregator_->Open(state));
+
+  RowBatch batch(child(0)->row_desc(), state->batch_size(), mem_tracker());
+  // Read all the rows from the child and process them.
+  bool eos = false;
+  do {
+    RETURN_IF_CANCELLED(state);
+    RETURN_IF_ERROR(QueryMaintenance(state));
+    RETURN_IF_ERROR(children_[0]->GetNext(state, &batch, &eos));
+    RETURN_IF_ERROR(aggregator_->AddBatch(state, &batch));
+    batch.Reset();
+  } while (!eos);
+
+  // The child can be closed at this point in most cases because we have consumed all of
+  // the input from the child and transfered ownership of the resources we need. The
+  // exception is if we are inside a subplan expecting to call Open()/GetNext() on the
+  // child again,
+  if (!IsInSubplan()) child(0)->Close(state);
+
+  RETURN_IF_ERROR(aggregator_->InputDone());
+  return Status::OK();
+}
+
+Status AggregationNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
+  SCOPED_TIMER(runtime_profile_->total_time_counter());
+  RETURN_IF_ERROR(ExecDebugAction(TExecNodePhase::GETNEXT, state));
+  RETURN_IF_CANCELLED(state);
+  RETURN_IF_ERROR(QueryMaintenance(state));
+
+  if (ReachedLimit()) {
+    *eos = true;
+    return Status::OK();
+  }
+
+  RETURN_IF_ERROR(aggregator_->GetNext(state, row_batch, eos));
+  num_rows_returned_ += row_batch->num_rows();
+  COUNTER_SET(rows_returned_counter_, num_rows_returned_);
+  return Status::OK();
+}
+
+Status AggregationNode::Reset(RuntimeState* state) {
+  RETURN_IF_ERROR(aggregator_->Reset(state));
+  return ExecNode::Reset(state);
+}
+
+void AggregationNode::Close(RuntimeState* state) {
+  if (is_closed()) return;
+  aggregator_->Close(state);
+  ExecNode::Close(state);
+}
+
+void AggregationNode::DebugString(int indentation_level, stringstream* out) const {
+  *out << string(indentation_level * 2, ' ');
+  *out << "AggregationNode("
+       << "aggregator=" << aggregator_->DebugString();
+  ExecNode::DebugString(indentation_level, out);
+  *out << ")";
+}
+} // namespace impala
diff --git a/be/src/exec/aggregation-node.h b/be/src/exec/aggregation-node.h
new file mode 100644
index 0000000..527a62d
--- /dev/null
+++ b/be/src/exec/aggregation-node.h
@@ -0,0 +1,60 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef IMPALA_EXEC_AGGREGATION_NODE_H
+#define IMPALA_EXEC_AGGREGATION_NODE_H
+
+#include <memory>
+
+#include "exec/aggregator.h"
+#include "exec/exec-node.h"
+
+namespace impala {
+
+class RowBatch;
+class RuntimeState;
+
+/// Node for doing partitioned hash aggregation.
+/// This node consumes the input from child(0) during Open() and then passes it to the
+/// Aggregator, which does the actual work of aggregating.
+class AggregationNode : public ExecNode {
+ public:
+  AggregationNode(ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
+
+  virtual Status Init(const TPlanNode& tnode, RuntimeState* state) override;
+  virtual Status Prepare(RuntimeState* state) override;
+  virtual void Codegen(RuntimeState* state) override;
+  virtual Status Open(RuntimeState* state) override;
+  virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
+  virtual Status Reset(RuntimeState* state) override;
+  virtual void Close(RuntimeState* state) override;
+
+  virtual void DebugString(int indentation_level, std::stringstream* out) const override;
+
+ private:
+  /////////////////////////////////////////
+  /// BEGIN: Members that must be Reset()
+
+  /// Performs the actual work of aggregating input rows.
+  std::unique_ptr<Aggregator> aggregator_;
+
+  /// END: Members that must be Reset()
+  /////////////////////////////////////////
+};
+} // namespace impala
+
+#endif // IMPALA_EXEC_AGGREGATION_NODE_H
diff --git a/be/src/exec/aggregator.cc b/be/src/exec/aggregator.cc
new file mode 100644
index 0000000..70178cc
--- /dev/null
+++ b/be/src/exec/aggregator.cc
@@ -0,0 +1,609 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/aggregator.h"
+
+#include <sstream>
+
+#include "codegen/codegen-anyval.h"
+#include "codegen/llvm-codegen.h"
+#include "exec/exec-node.h"
+#include "exprs/agg-fn-evaluator.h"
+#include "exprs/expr-value.h"
+#include "exprs/scalar-expr.h"
+#include "gutil/strings/substitute.h"
+#include "runtime/descriptors.h"
+#include "runtime/mem-pool.h"
+#include "runtime/mem-tracker.h"
+#include "runtime/raw-value.h"
+#include "runtime/runtime-state.h"
+#include "runtime/tuple-row.h"
+#include "runtime/tuple.h"
+#include "util/runtime-profile-counters.h"
+
+#include "gen-cpp/PlanNodes_types.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+const char* Aggregator::LLVM_CLASS_NAME = "class.impala::Aggregator";
+
+Aggregator::Aggregator(ExecNode* exec_node, ObjectPool* pool, const TPlanNode& tnode,
+    const DescriptorTbl& descs, const std::string& name)
+  : id_(exec_node->id()),
+    pool_(pool),
+    intermediate_tuple_id_(tnode.agg_node.intermediate_tuple_id),
+    intermediate_tuple_desc_(descs.GetTupleDescriptor(intermediate_tuple_id_)),
+    output_tuple_id_(tnode.agg_node.output_tuple_id),
+    output_tuple_desc_(descs.GetTupleDescriptor(output_tuple_id_)),
+    row_desc_(*exec_node->row_desc()),
+    input_row_desc_(*exec_node->child(0)->row_desc()),
+    needs_finalize_(tnode.agg_node.need_finalize),
+    runtime_profile_(RuntimeProfile::Create(pool_, name)),
+    num_rows_returned_(0),
+    rows_returned_counter_(nullptr),
+    build_timer_(nullptr) {}
+
+Aggregator::~Aggregator() {}
+
+Status Aggregator::Init(const TPlanNode& tnode, RuntimeState* state) {
+  DCHECK(intermediate_tuple_desc_ != nullptr);
+  DCHECK(output_tuple_desc_ != nullptr);
+  DCHECK_EQ(intermediate_tuple_desc_->slots().size(), output_tuple_desc_->slots().size());
+
+  int j = num_grouping_exprs();
+  for (int i = 0; i < tnode.agg_node.aggregate_functions.size(); ++i, ++j) {
+    SlotDescriptor* intermediate_slot_desc = intermediate_tuple_desc_->slots()[j];
+    SlotDescriptor* output_slot_desc = output_tuple_desc_->slots()[j];
+    AggFn* agg_fn;
+    RETURN_IF_ERROR(AggFn::Create(tnode.agg_node.aggregate_functions[i], input_row_desc_,
+        *intermediate_slot_desc, *output_slot_desc, state, &agg_fn));
+    agg_fns_.push_back(agg_fn);
+  }
+
+  RETURN_IF_ERROR(ScalarExpr::Create(tnode.conjuncts, row_desc_, state, &conjuncts_));
+  return Status::OK();
+}
+
+Status Aggregator::Prepare(RuntimeState* state) {
+  mem_tracker_.reset(new MemTracker(
+      runtime_profile_, -1, runtime_profile_->name(), state->instance_mem_tracker()));
+  expr_mem_tracker_.reset(new MemTracker(-1, "Exprs", mem_tracker_.get(), false));
+  expr_perm_pool_.reset(new MemPool(expr_mem_tracker_.get()));
+  expr_results_pool_.reset(new MemPool(expr_mem_tracker_.get()));
+
+  RETURN_IF_ERROR(AggFnEvaluator::Create(agg_fns_, state, pool_, expr_perm_pool_.get(),
+      expr_results_pool_.get(), &agg_fn_evals_));
+  RETURN_IF_ERROR(ScalarExprEvaluator::Create(conjuncts_, state, pool_,
+      expr_perm_pool_.get(), expr_results_pool_.get(), &conjunct_evals_));
+  DCHECK_EQ(conjunct_evals_.size(), conjuncts_.size());
+
+  rows_returned_counter_ = ADD_COUNTER(runtime_profile_, "RowsReturned", TUnit::UNIT);
+  build_timer_ = ADD_TIMER(runtime_profile(), "BuildTime");
+
+  return Status::OK();
+}
+
+Status Aggregator::Open(RuntimeState* state) {
+  RETURN_IF_ERROR(AggFnEvaluator::Open(agg_fn_evals_, state));
+  RETURN_IF_ERROR(ScalarExprEvaluator::Open(conjunct_evals_, state));
+  return Status::OK();
+}
+
+void Aggregator::Close(RuntimeState* state) {
+  // Close all the agg-fn-evaluators
+  AggFnEvaluator::Close(agg_fn_evals_, state);
+  AggFn::Close(agg_fns_);
+  ScalarExprEvaluator::Close(conjunct_evals_, state);
+  ScalarExpr::Close(conjuncts_);
+
+  if (expr_perm_pool_.get() != nullptr) expr_perm_pool_->FreeAll();
+  if (expr_results_pool_.get() != nullptr) expr_results_pool_->FreeAll();
+  if (expr_mem_tracker_.get() != nullptr) expr_mem_tracker_->Close();
+  if (mem_tracker_.get() != nullptr) mem_tracker_->Close();
+}
+
+// TODO: codegen this function.
+void Aggregator::InitAggSlots(
+    const vector<AggFnEvaluator*>& agg_fn_evals, Tuple* intermediate_tuple) {
+  vector<SlotDescriptor*>::const_iterator slot_desc =
+      intermediate_tuple_desc_->slots().begin() + num_grouping_exprs();
+  for (int i = 0; i < agg_fn_evals.size(); ++i, ++slot_desc) {
+    // To minimize branching on the UpdateTuple path, initialize the result value so that
+    // the Add() UDA function can ignore the NULL bit of its destination value. E.g. for
+    // SUM(), if we initialize the destination value to 0 (with the NULL bit set), we can
+    // just start adding to the destination value (rather than repeatedly checking the
+    // destination NULL bit. The codegen'd version of UpdateSlot() exploits this to
+    // eliminate a branch per value.
+    //
+    // For boolean and numeric types, the default values are false/0, so the nullable
+    // aggregate functions SUM() and AVG() produce the correct result. For MIN()/MAX(),
+    // initialize the value to max/min possible value for the same effect.
+    AggFnEvaluator* eval = agg_fn_evals[i];
+    eval->Init(intermediate_tuple);
+
+    DCHECK(agg_fns_[i] == &(eval->agg_fn()));
+    const AggFn* agg_fn = agg_fns_[i];
+    const AggFn::AggregationOp agg_op = agg_fn->agg_op();
+    if ((agg_op == AggFn::MIN || agg_op == AggFn::MAX)
+        && !agg_fn->intermediate_type().IsStringType()
+        && !agg_fn->intermediate_type().IsTimestampType()) {
+      ExprValue default_value;
+      void* default_value_ptr = nullptr;
+      if (agg_op == AggFn::MIN) {
+        default_value_ptr = default_value.SetToMax((*slot_desc)->type());
+      } else {
+        DCHECK_EQ(agg_op, AggFn::MAX);
+        default_value_ptr = default_value.SetToMin((*slot_desc)->type());
+      }
+      RawValue::Write(default_value_ptr, intermediate_tuple, *slot_desc, nullptr);
+    }
+  }
+}
+
+void Aggregator::UpdateTuple(
+    AggFnEvaluator** agg_fn_evals, Tuple* tuple, TupleRow* row, bool is_merge) noexcept {
+  DCHECK(tuple != nullptr || agg_fns_.empty());
+  for (int i = 0; i < agg_fns_.size(); ++i) {
+    if (is_merge) {
+      agg_fn_evals[i]->Merge(row->GetTuple(0), tuple);
+    } else {
+      agg_fn_evals[i]->Add(row, tuple);
+    }
+  }
+}
+
+Tuple* Aggregator::GetOutputTuple(
+    const vector<AggFnEvaluator*>& agg_fn_evals, Tuple* tuple, MemPool* pool) {
+  DCHECK(tuple != nullptr || agg_fn_evals.empty()) << tuple;
+  Tuple* dst = tuple;
+  if (needs_finalize_ && intermediate_tuple_id_ != output_tuple_id_) {
+    dst = Tuple::Create(output_tuple_desc_->byte_size(), pool);
+  }
+  if (needs_finalize_) {
+    AggFnEvaluator::Finalize(agg_fn_evals, tuple, dst);
+  } else {
+    AggFnEvaluator::Serialize(agg_fn_evals, tuple);
+  }
+  // Copy grouping values from tuple to dst.
+  // TODO: Codegen this.
+  if (dst != tuple) {
+    int num_grouping_slots = num_grouping_exprs();
+    for (int i = 0; i < num_grouping_slots; ++i) {
+      SlotDescriptor* src_slot_desc = intermediate_tuple_desc_->slots()[i];
+      SlotDescriptor* dst_slot_desc = output_tuple_desc_->slots()[i];
+      bool src_slot_null = tuple->IsNull(src_slot_desc->null_indicator_offset());
+      void* src_slot = nullptr;
+      if (!src_slot_null) src_slot = tuple->GetSlot(src_slot_desc->tuple_offset());
+      RawValue::Write(src_slot, dst, dst_slot_desc, nullptr);
+    }
+  }
+  return dst;
+}
+
+// IR Generation for updating a single aggregation slot. Signature is:
+// void UpdateSlot(AggFnEvaluator* agg_expr_eval, AggTuple* agg_tuple, char** row)
+//
+// The IR for sum(double_col), which is constructed directly with the IRBuilder, is:
+//
+// define void @UpdateSlot(%"class.impala::AggFnEvaluator"* %agg_fn_eval,
+//     <{ double, i8 }>* %agg_tuple, %"class.impala::TupleRow"* %row) #33 {
+// entry:
+//   %input_evals_vector = call %"class.impala::ScalarExprEvaluator"**
+//       @_ZNK6impala14AggFnEvaluator11input_evalsEv(
+//           %"class.impala::AggFnEvaluator"* %agg_fn_eval)
+//   %0 = getelementptr %"class.impala::ScalarExprEvaluator"*,
+//       %"class.impala::ScalarExprEvaluator"** %input_evals_vector, i32 0
+//   %input_eval = load %"class.impala::ScalarExprEvaluator"*,
+//       %"class.impala::ScalarExprEvaluator"** %0
+//   %input0 = call { i8, double } @GetSlotRef(%"class.impala::ScalarExprEvaluator"*
+//       %input_eval, %"class.impala::TupleRow"* %row)
+//   %dst_slot_ptr = getelementptr inbounds <{ double, i8 }>,
+//       <{ double, i8 }>* %agg_tuple, i32 0, i32 0
+//   %dst_val = load double, double* %dst_slot_ptr
+//   %1 = extractvalue { i8, double } %input0, 0
+//   %is_null = trunc i8 %1 to i1
+//   br i1 %is_null, label %ret, label %not_null
+//
+// ret:                                              ; preds = %not_null, %entry
+//   ret void
+//
+// not_null:                                         ; preds = %entry
+//   %val = extractvalue { i8, double } %input0, 1
+//   %2 = fadd double %dst_val, %val
+//   %3 = bitcast <{ double, i8 }>* %agg_tuple to i8*
+//   %null_byte_ptr = getelementptr inbounds i8, i8* %3, i32 8
+//   %null_byte = load i8, i8* %null_byte_ptr
+//   %null_bit_cleared = and i8 %null_byte, -2
+//   store i8 %null_bit_cleared, i8* %null_byte_ptr
+//   store double %2, double* %dst_slot_ptr
+//   br label %ret
+// }
+//
+// The IR for ndv(timestamp_col), which uses the UDA interface, is:
+//
+// define void @UpdateSlot(%"class.impala::AggFnEvaluator"* %agg_fn_eval,
+//     <{ [1024 x i8] }>* %agg_tuple,
+//     %"class.impala::TupleRow"* %row) #39 {
+// entry:
+//   %dst_lowered_ptr = alloca { i64, i8* }
+//   %0 = alloca { i64, i64 }
+//   %input_evals_vector = call %"class.impala::ScalarExprEvaluator"**
+//       @_ZNK6impala14AggFnEvaluator11input_evalsEv(
+//           %"class.impala::AggFnEvaluator"* %agg_fn_eval)
+//   %1 = getelementptr %"class.impala::ScalarExprEvaluator"*,
+//       %"class.impala::ScalarExprEvaluator"** %input_evals_vector, i32 0
+//   %input_eval = load %"class.impala::ScalarExprEvaluator"*,
+//       %"class.impala::ScalarExprEvaluator"** %1
+//   %input0 = call { i64, i64 } @GetSlotRef(
+//       %"class.impala::ScalarExprEvaluator"* %input_eval,
+//       %"class.impala::TupleRow"* %row)
+//   %dst_slot_ptr = getelementptr inbounds <{ [1024 x i8] }>,
+//       <{ [1024 x i8] }>* %agg_tuple, i32 0, i32 0
+//   %2 = bitcast [1024 x i8]* %dst_slot_ptr to i8*
+//   %dst = insertvalue { i64, i8* } zeroinitializer, i8* %2, 1
+//   %3 = extractvalue { i64, i8* } %dst, 0
+//   %4 = and i64 %3, 4294967295
+//   %5 = or i64 %4, 4398046511104
+//   %dst1 = insertvalue { i64, i8* } %dst, i64 %5, 0
+//   %agg_fn_ctx = call %"class.impala_udf::FunctionContext"*
+//       @_ZNK6impala14AggFnEvaluator10agg_fn_ctxEv(
+//          %"class.impala::AggFnEvaluator"* %agg_fn_eval)
+//   store { i64, i64 } %input0, { i64, i64 }* %0
+//   %input_unlowered_ptr =
+//       bitcast { i64, i64 }* %0 to %"struct.impala_udf::TimestampVal"*
+//   store { i64, i8* } %dst1, { i64, i8* }* %dst_lowered_ptr
+//   %dst_unlowered_ptr =
+//       bitcast { i64, i8* }* %dst_lowered_ptr to %"struct.impala_udf::StringVal"*
+//   call void @"void impala::AggregateFunctions::HllUpdate<impala_udf::TimestampVal>"(
+//       %"class.impala_udf::FunctionContext"* %agg_fn_ctx,
+//       %"struct.impala_udf::TimestampVal"* %input_unlowered_ptr,
+//       %"struct.impala_udf::StringVal"* %dst_unlowered_ptr)
+//   %anyval_result = load { i64, i8* }, { i64, i8* }* %dst_lowered_ptr
+//   br label %ret
+//
+// ret:                                              ; preds = %entry
+//   ret void
+// }
+//
+Status Aggregator::CodegenUpdateSlot(LlvmCodeGen* codegen, int agg_fn_idx,
+    SlotDescriptor* slot_desc, llvm::Function** fn) {
+  llvm::PointerType* agg_fn_eval_type = codegen->GetStructPtrType<AggFnEvaluator>();
+  llvm::StructType* tuple_struct = intermediate_tuple_desc_->GetLlvmStruct(codegen);
+  if (tuple_struct == nullptr) {
+    return Status("Aggregator::CodegenUpdateSlot(): failed to generate "
+                  "intermediate tuple desc");
+  }
+  llvm::PointerType* tuple_ptr_type = codegen->GetPtrType(tuple_struct);
+  llvm::PointerType* tuple_row_ptr_type = codegen->GetStructPtrType<TupleRow>();
+
+  LlvmCodeGen::FnPrototype prototype(codegen, "UpdateSlot", codegen->void_type());
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("agg_fn_eval", agg_fn_eval_type));
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("agg_tuple", tuple_ptr_type));
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("row", tuple_row_ptr_type));
+
+  LlvmBuilder builder(codegen->context());
+  llvm::Value* args[3];
+  *fn = prototype.GeneratePrototype(&builder, &args[0]);
+  llvm::Value* agg_fn_eval_arg = args[0];
+  llvm::Value* agg_tuple_arg = args[1];
+  llvm::Value* row_arg = args[2];
+
+  // Get the vector of input expressions' evaluators.
+  llvm::Value* input_evals_vector = codegen->CodegenCallFunction(&builder,
+      IRFunction::AGG_FN_EVALUATOR_INPUT_EVALUATORS, agg_fn_eval_arg,
+      "input_evals_vector");
+
+  AggFn* agg_fn = agg_fns_[agg_fn_idx];
+  const int num_inputs = agg_fn->GetNumChildren();
+  DCHECK_GE(num_inputs, 1);
+  vector<CodegenAnyVal> input_vals;
+  for (int i = 0; i < num_inputs; ++i) {
+    ScalarExpr* input_expr = agg_fn->GetChild(i);
+    llvm::Function* input_expr_fn;
+    RETURN_IF_ERROR(input_expr->GetCodegendComputeFn(codegen, &input_expr_fn));
+    DCHECK(input_expr_fn != nullptr);
+
+    // Call input expr function with the matching evaluator to get src slot value.
+    llvm::Value* input_eval =
+        codegen->CodegenArrayAt(&builder, input_evals_vector, i, "input_eval");
+    string input_name = Substitute("input$0", i);
+    CodegenAnyVal input_val = CodegenAnyVal::CreateCallWrapped(codegen, &builder,
+        input_expr->type(), input_expr_fn,
+        llvm::ArrayRef<llvm::Value*>({input_eval, row_arg}), input_name.c_str());
+    input_vals.push_back(input_val);
+  }
+
+  AggFn::AggregationOp agg_op = agg_fn->agg_op();
+  const ColumnType& dst_type = agg_fn->intermediate_type();
+  bool dst_is_int_or_float_or_bool = dst_type.IsIntegerType()
+      || dst_type.IsFloatingPointType() || dst_type.IsBooleanType();
+  bool dst_is_numeric_or_bool = dst_is_int_or_float_or_bool || dst_type.IsDecimalType();
+
+  llvm::BasicBlock* ret_block = llvm::BasicBlock::Create(codegen->context(), "ret", *fn);
+
+  // Emit the code to compute 'result' and set the NULL indicator if needed. First check
+  // for special cases where we can emit a very simple instruction sequence, then fall
+  // back to the general-purpose approach of calling the cross-compiled builtin UDA.
+  CodegenAnyVal& src = input_vals[0];
+
+  // 'dst_slot_ptr' points to the slot in the aggregate tuple to update.
+  llvm::Value* dst_slot_ptr = builder.CreateStructGEP(
+      nullptr, agg_tuple_arg, slot_desc->llvm_field_idx(), "dst_slot_ptr");
+  // TODO: consider moving the following codegen logic to AggFn.
+  if (agg_op == AggFn::COUNT) {
+    src.CodegenBranchIfNull(&builder, ret_block);
+    llvm::Value* dst_value = builder.CreateLoad(dst_slot_ptr, "dst_val");
+    llvm::Value* result = agg_fn->is_merge() ?
+        builder.CreateAdd(dst_value, src.GetVal(), "count_sum") :
+        builder.CreateAdd(dst_value, codegen->GetI64Constant(1), "count_inc");
+    builder.CreateStore(result, dst_slot_ptr);
+    DCHECK(!slot_desc->is_nullable());
+  } else if ((agg_op == AggFn::MIN || agg_op == AggFn::MAX) && dst_is_numeric_or_bool) {
+    bool is_min = agg_op == AggFn::MIN;
+    src.CodegenBranchIfNull(&builder, ret_block);
+    codegen->CodegenMinMax(
+        &builder, slot_desc->type(), src.GetVal(), dst_slot_ptr, is_min, *fn);
+
+    // Dst may have been NULL, make sure to unset the NULL bit.
+    DCHECK(slot_desc->is_nullable());
+    slot_desc->CodegenSetNullIndicator(
+        codegen, &builder, agg_tuple_arg, codegen->false_value());
+  } else if (agg_op == AggFn::SUM && dst_is_int_or_float_or_bool) {
+    src.CodegenBranchIfNull(&builder, ret_block);
+    llvm::Value* dst_value = builder.CreateLoad(dst_slot_ptr, "dst_val");
+    llvm::Value* result = dst_type.IsFloatingPointType() ?
+        builder.CreateFAdd(dst_value, src.GetVal()) :
+        builder.CreateAdd(dst_value, src.GetVal());
+    builder.CreateStore(result, dst_slot_ptr);
+
+    if (slot_desc->is_nullable()) {
+      slot_desc->CodegenSetNullIndicator(
+          codegen, &builder, agg_tuple_arg, codegen->false_value());
+    } else {
+      // 'slot_desc' is not nullable if the aggregate function is sum_init_zero(),
+      // because the slot is initialized to be zero and the null bit is nonexistent.
+      DCHECK_EQ(agg_fn->fn_name(), "sum_init_zero");
+    }
+  } else {
+    // The remaining cases are implemented using the UDA interface.
+    // Create intermediate argument 'dst' from 'dst_value'
+    CodegenAnyVal dst = CodegenAnyVal::GetNonNullVal(codegen, &builder, dst_type, "dst");
+
+    // For a subset of builtins we generate a different code sequence that exploits two
+    // properties of the builtins. First, NULL input values can be skipped. Second, the
+    // value of the slot was initialized in the right way in InitAggSlots() (e.g. 0 for
+    // SUM) that we get the right result if UpdateSlot() pretends that the NULL bit of
+    // 'dst' is unset. Empirically this optimisation makes TPC-H Q1 5-10% faster.
+    bool special_null_handling = !agg_fn->intermediate_type().IsStringType()
+        && !agg_fn->intermediate_type().IsTimestampType()
+        && (agg_op == AggFn::MIN || agg_op == AggFn::MAX || agg_op == AggFn::SUM
+               || agg_op == AggFn::AVG || agg_op == AggFn::NDV);
+    if (slot_desc->is_nullable()) {
+      if (special_null_handling) {
+        src.CodegenBranchIfNull(&builder, ret_block);
+        slot_desc->CodegenSetNullIndicator(
+            codegen, &builder, agg_tuple_arg, codegen->false_value());
+      } else {
+        dst.SetIsNull(slot_desc->CodegenIsNull(codegen, &builder, agg_tuple_arg));
+      }
+    }
+    dst.LoadFromNativePtr(dst_slot_ptr);
+
+    // Get the FunctionContext object for the AggFnEvaluator.
+    llvm::Function* get_agg_fn_ctx_fn =
+        codegen->GetFunction(IRFunction::AGG_FN_EVALUATOR_AGG_FN_CTX, false);
+    DCHECK(get_agg_fn_ctx_fn != nullptr);
+    llvm::Value* agg_fn_ctx_val =
+        builder.CreateCall(get_agg_fn_ctx_fn, {agg_fn_eval_arg}, "agg_fn_ctx");
+
+    // Call the UDA to update/merge 'src' into 'dst', with the result stored in
+    // 'updated_dst_val'.
+    CodegenAnyVal updated_dst_val;
+    RETURN_IF_ERROR(CodegenCallUda(
+        codegen, &builder, agg_fn, agg_fn_ctx_val, input_vals, dst, &updated_dst_val));
+    // Copy the value back to the slot. In the FIXED_UDA_INTERMEDIATE case, the
+    // UDA function writes directly to the slot so there is nothing to copy.
+    if (dst_type.type != TYPE_FIXED_UDA_INTERMEDIATE) {
+      updated_dst_val.StoreToNativePtr(dst_slot_ptr);
+    }
+
+    if (slot_desc->is_nullable() && !special_null_handling) {
+      // Set NULL bit in the slot based on the return value.
+      llvm::Value* result_is_null = updated_dst_val.GetIsNull("result_is_null");
+      slot_desc->CodegenSetNullIndicator(
+          codegen, &builder, agg_tuple_arg, result_is_null);
+    }
+  }
+  builder.CreateBr(ret_block);
+
+  builder.SetInsertPoint(ret_block);
+  builder.CreateRetVoid();
+
+  // Avoid producing huge UpdateTuple() function after inlining - LLVM's optimiser
+  // memory/CPU usage scales super-linearly with function size.
+  // E.g. compute stats on all columns of a 1000-column table previously took 4 minutes to
+  // codegen because all the UpdateSlot() functions were inlined.
+  if (agg_fn_idx >= LlvmCodeGen::CODEGEN_INLINE_EXPRS_THRESHOLD) {
+    codegen->SetNoInline(*fn);
+  }
+
+  *fn = codegen->FinalizeFunction(*fn);
+  if (*fn == nullptr) {
+    return Status("Aggregator::CodegenUpdateSlot(): codegen'd "
+                  "UpdateSlot() function failed verification, see log");
+  }
+  return Status::OK();
+}
+
+Status Aggregator::CodegenCallUda(LlvmCodeGen* codegen, LlvmBuilder* builder,
+    AggFn* agg_fn, llvm::Value* agg_fn_ctx_val, const vector<CodegenAnyVal>& input_vals,
+    const CodegenAnyVal& dst_val, CodegenAnyVal* updated_dst_val) {
+  llvm::Function* uda_fn;
+  RETURN_IF_ERROR(agg_fn->CodegenUpdateOrMergeFunction(codegen, &uda_fn));
+
+  // Set up arguments for call to UDA, which are the FunctionContext*, followed by
+  // pointers to all input values, followed by a pointer to the destination value.
+  vector<llvm::Value*> uda_fn_args;
+  uda_fn_args.push_back(agg_fn_ctx_val);
+
+  // Create pointers to input args to pass to uda_fn. We must use the unlowered type,
+  // e.g. IntVal, because the UDA interface expects the values to be passed as const
+  // references to the classes.
+  DCHECK_EQ(agg_fn->GetNumChildren(), input_vals.size());
+  for (int i = 0; i < input_vals.size(); ++i) {
+    uda_fn_args.push_back(input_vals[i].GetUnloweredPtr("input_unlowered_ptr"));
+  }
+
+  // Create pointer to dst to pass to uda_fn. We must use the unlowered type for the
+  // same reason as above.
+  llvm::Value* dst_lowered_ptr = dst_val.GetLoweredPtr("dst_lowered_ptr");
+  const ColumnType& dst_type = agg_fn->intermediate_type();
+  llvm::Type* dst_unlowered_ptr_type =
+      CodegenAnyVal::GetUnloweredPtrType(codegen, dst_type);
+  llvm::Value* dst_unlowered_ptr = builder->CreateBitCast(
+      dst_lowered_ptr, dst_unlowered_ptr_type, "dst_unlowered_ptr");
+  uda_fn_args.push_back(dst_unlowered_ptr);
+
+  // Call 'uda_fn'
+  builder->CreateCall(uda_fn, uda_fn_args);
+
+  // Convert intermediate 'dst_arg' back to the native type.
+  llvm::Value* anyval_result = builder->CreateLoad(dst_lowered_ptr, "anyval_result");
+
+  *updated_dst_val = CodegenAnyVal(codegen, builder, dst_type, anyval_result);
+  return Status::OK();
+}
+
+// IR codegen for the UpdateTuple loop.  This loop is query specific and based on the
+// aggregate functions.  The function signature must match the non- codegen'd UpdateTuple
+// exactly.
+// For the query:
+// select count(*), count(int_col), sum(double_col) the IR looks like:
+//
+// define void @UpdateTuple(%"class.impala::Aggregator"* %this_ptr,
+//     %"class.impala::AggFnEvaluator"** %agg_fn_evals, %"class.impala::Tuple"* %tuple,
+//     %"class.impala::TupleRow"* %row, i1 %is_merge) #33 {
+// entry:
+//   %tuple1 = bitcast %"class.impala::Tuple"* %tuple to <{ i64, i64, double, i8 }>*
+//   %src_slot = getelementptr inbounds <{ i64, i64, double, i8 }>,
+//       <{ i64, i64, double, i8 }>* %tuple1, i32 0, i32 0
+//   %count_star_val = load i64, i64* %src_slot
+//   %count_star_inc = add i64 %count_star_val, 1
+//   store i64 %count_star_inc, i64* %src_slot
+//   %0 = getelementptr %"class.impala::AggFnEvaluator"*,
+//       %"class.impala::AggFnEvaluator"** %agg_fn_evals, i32 1
+//   %agg_fn_eval =
+//       load %"class.impala::AggFnEvaluator"*, %"class.impala::AggFnEvaluator"** %0
+//   call void @UpdateSlot(%"class.impala::AggFnEvaluator"* %agg_fn_eval,
+//       <{ i64, i64, double, i8 }>* %tuple1, %"class.impala::TupleRow"* %row)
+//   %1 = getelementptr %"class.impala::AggFnEvaluator"*,
+//       %"class.impala::AggFnEvaluator"** %agg_fn_evals, i32 2
+//   %agg_fn_eval2 =
+//       load %"class.impala::AggFnEvaluator"*, %"class.impala::AggFnEvaluator"** %1
+//   call void @UpdateSlot.2(%"class.impala::AggFnEvaluator"* %agg_fn_eval2,
+//       <{ i64, i64, double, i8 }>* %tuple1, %"class.impala::TupleRow"* %row)
+//   ret void
+// }
+//
+Status Aggregator::CodegenUpdateTuple(LlvmCodeGen* codegen, llvm::Function** fn) {
+  for (const SlotDescriptor* slot_desc : intermediate_tuple_desc_->slots()) {
+    if (slot_desc->type().type == TYPE_CHAR) {
+      return Status::Expected("Aggregator::CodegenUpdateTuple(): cannot "
+                              "codegen CHAR in aggregations");
+    }
+  }
+
+  if (intermediate_tuple_desc_->GetLlvmStruct(codegen) == nullptr) {
+    return Status::Expected("Aggregator::CodegenUpdateTuple(): failed to"
+                            " generate intermediate tuple desc");
+  }
+
+  // Get the types to match the UpdateTuple signature
+  llvm::PointerType* agg_node_ptr_type = codegen->GetStructPtrType<Aggregator>();
+  llvm::PointerType* evals_type = codegen->GetStructPtrPtrType<AggFnEvaluator>();
+  llvm::PointerType* tuple_ptr_type = codegen->GetStructPtrType<Tuple>();
+  llvm::PointerType* tuple_row_ptr_type = codegen->GetStructPtrType<TupleRow>();
+
+  llvm::StructType* tuple_struct = intermediate_tuple_desc_->GetLlvmStruct(codegen);
+  llvm::PointerType* tuple_ptr = codegen->GetPtrType(tuple_struct);
+  LlvmCodeGen::FnPrototype prototype(codegen, "UpdateTuple", codegen->void_type());
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("this_ptr", agg_node_ptr_type));
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("agg_fn_evals", evals_type));
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("tuple", tuple_ptr_type));
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("row", tuple_row_ptr_type));
+  prototype.AddArgument(LlvmCodeGen::NamedVariable("is_merge", codegen->bool_type()));
+
+  LlvmBuilder builder(codegen->context());
+  llvm::Value* args[5];
+  *fn = prototype.GeneratePrototype(&builder, &args[0]);
+  llvm::Value* agg_fn_evals_arg = args[1];
+  llvm::Value* tuple_arg = args[2];
+  llvm::Value* row_arg = args[3];
+
+  // Cast the parameter types to the internal llvm runtime types.
+  // TODO: get rid of this by using right type in function signature
+  tuple_arg = builder.CreateBitCast(tuple_arg, tuple_ptr, "tuple");
+
+  // Loop over each expr and generate the IR for that slot.  If the expr is not
+  // count(*), generate a helper IR function to update the slot and call that.
+  int j = num_grouping_exprs();
+  for (int i = 0; i < agg_fns_.size(); ++i, ++j) {
+    SlotDescriptor* slot_desc = intermediate_tuple_desc_->slots()[j];
+    AggFn* agg_fn = agg_fns_[i];
+    if (agg_fn->is_count_star()) {
+      // TODO: we should be able to hoist this up to the loop over the batch and just
+      // increment the slot by the number of rows in the batch.
+      int field_idx = slot_desc->llvm_field_idx();
+      llvm::Value* const_one = codegen->GetI64Constant(1);
+      llvm::Value* slot_ptr =
+          builder.CreateStructGEP(nullptr, tuple_arg, field_idx, "src_slot");
+      llvm::Value* slot_loaded = builder.CreateLoad(slot_ptr, "count_star_val");
+      llvm::Value* count_inc =
+          builder.CreateAdd(slot_loaded, const_one, "count_star_inc");
+      builder.CreateStore(count_inc, slot_ptr);
+    } else {
+      llvm::Function* update_slot_fn;
+      RETURN_IF_ERROR(CodegenUpdateSlot(codegen, i, slot_desc, &update_slot_fn));
+
+      // Load agg_fn_evals_[i]
+      llvm::Value* agg_fn_eval_val =
+          codegen->CodegenArrayAt(&builder, agg_fn_evals_arg, i, "agg_fn_eval");
+
+      // Call UpdateSlot(agg_fn_evals_[i], tuple, row);
+      llvm::Value* update_slot_args[] = {agg_fn_eval_val, tuple_arg, row_arg};
+      builder.CreateCall(update_slot_fn, update_slot_args);
+    }
+  }
+  builder.CreateRetVoid();
+
+  // Avoid inlining big UpdateTuple function into outer loop - we're unlikely to get
+  // any benefit from it since the function call overhead will be amortized.
+  if (agg_fns_.size() > LlvmCodeGen::CODEGEN_INLINE_EXPR_BATCH_THRESHOLD) {
+    codegen->SetNoInline(*fn);
+  }
+
+  // CodegenProcessBatch() does the final optimizations.
+  *fn = codegen->FinalizeFunction(*fn);
+  if (*fn == nullptr) {
+    return Status("Aggregator::CodegenUpdateTuple(): codegen'd "
+                  "UpdateTuple() function failed verification, see log");
+  }
+  return Status::OK();
+}
+} // namespace impala
diff --git a/be/src/exec/aggregator.h b/be/src/exec/aggregator.h
new file mode 100644
index 0000000..ab13d45
--- /dev/null
+++ b/be/src/exec/aggregator.h
@@ -0,0 +1,211 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef IMPALA_EXEC_AGGREGATOR_H
+#define IMPALA_EXEC_AGGREGATOR_H
+
+#include <vector>
+
+#include "common/global-types.h"
+#include "common/status.h"
+#include "gen-cpp/Types_types.h"
+#include "util/runtime-profile.h"
+
+namespace llvm {
+class Function;
+class Value;
+} // namespace llvm
+
+namespace impala {
+
+class AggFn;
+class AggFnEvaluator;
+class CodegenAnyVal;
+class DescriptorTbl;
+class ExecNode;
+class LlvmBuilder;
+class LlvmCodeGen;
+class MemPool;
+class MemTracker;
+class ObjectPool;
+class RowBatch;
+class RowDescriptor;
+class RuntimeState;
+class ScalarExpr;
+class ScalarExprEvaluator;
+class SlotDescriptor;
+class TPlanNode;
+class Tuple;
+class TupleDescriptor;
+class TupleRow;
+
+/// Base class for aggregating rows. Used in the AggregationNode and
+/// StreamingAggregationNode.
+///
+/// Rows are added by calling AddBatch(). Once all rows have been added, InputDone() must
+/// be called and the results can be fetched with GetNext().
+class Aggregator {
+ public:
+  Aggregator(ExecNode* exec_node, ObjectPool* pool, const TPlanNode& tnode,
+      const DescriptorTbl& descs, const std::string& name);
+  virtual ~Aggregator();
+
+  /// Aggregators follow the same lifecycle as ExecNodes, except that after Open() and
+  /// before GetNext() rows should be added with AddBatch(), followed by InputDone()[
+  virtual Status Init(const TPlanNode& tnode, RuntimeState* state) WARN_UNUSED_RESULT;
+  virtual Status Prepare(RuntimeState* state) WARN_UNUSED_RESULT;
+  virtual void Codegen(RuntimeState* state) = 0;
+  virtual Status Open(RuntimeState* state) WARN_UNUSED_RESULT;
+  virtual Status GetNext(
+      RuntimeState* state, RowBatch* row_batch, bool* eos) WARN_UNUSED_RESULT = 0;
+  virtual Status Reset(RuntimeState* state) WARN_UNUSED_RESULT = 0;
+  virtual void Close(RuntimeState* state);
+
+  /// Adds all of the rows in 'batch' to the aggregation.
+  virtual Status AddBatch(RuntimeState* state, RowBatch* batch) = 0;
+  /// Indicates that all batches have been added. Must be called before GetNext().
+  virtual Status InputDone() = 0;
+
+  virtual int num_grouping_exprs() = 0;
+  RuntimeProfile* runtime_profile() { return runtime_profile_; }
+
+  virtual void SetDebugOptions(const TDebugOptions& debug_options) = 0;
+
+  virtual std::string DebugString(int indentation_level = 0) const = 0;
+  virtual void DebugString(int indentation_level, std::stringstream* out) const = 0;
+
+  static const char* LLVM_CLASS_NAME;
+
+ protected:
+  /// The id of the ExecNode this Aggregator corresponds to.
+  int id_;
+  ObjectPool* pool_;
+
+  /// Account for peak memory used by this aggregator.
+  std::unique_ptr<MemTracker> mem_tracker_;
+
+  /// MemTracker used by 'expr_perm_pool_' and 'expr_results_pool_'.
+  std::unique_ptr<MemTracker> expr_mem_tracker_;
+
+  /// MemPool for allocations made by expression evaluators in this aggregator that are
+  /// "permanent" and live until Close() is called. Created in Prepare().
+  std::unique_ptr<MemPool> expr_perm_pool_;
+
+  /// MemPool for allocations made by expression evaluators in this aggregator that hold
+  /// intermediate or final results of expression evaluation. Should be cleared
+  /// periodically to free accumulated memory. QueryMaintenance() clears this pool, but
+  /// it may be appropriate for Aggregator implementation to clear it at other points in
+  /// execution where the memory is not needed.
+  std::unique_ptr<MemPool> expr_results_pool_;
+
+  /// Tuple into which Update()/Merge()/Serialize() results are stored.
+  TupleId intermediate_tuple_id_;
+  TupleDescriptor* intermediate_tuple_desc_;
+
+  /// Tuple into which Finalize() results are stored. Possibly the same as
+  /// the intermediate tuple.
+  TupleId output_tuple_id_;
+  TupleDescriptor* output_tuple_desc_;
+
+  /// The RowDescriptor for the exec node this aggregator corresponds to.
+  const RowDescriptor& row_desc_;
+  /// The RowDescriptor for the child of the exec node this aggregator corresponds to.
+  const RowDescriptor& input_row_desc_;
+
+  /// Certain aggregates require a finalize step, which is the final step of the
+  /// aggregate after consuming all input rows. The finalize step converts the aggregate
+  /// value into its final form. This is true if this aggregator contains aggregate that
+  /// requires a finalize step.
+  const bool needs_finalize_;
+
+  /// The list of all aggregate operations for this aggregator.
+  std::vector<AggFn*> agg_fns_;
+
+  /// Evaluators for each aggregate function. If this is a grouping aggregation, these
+  /// evaluators are only used to create cloned per-partition evaluators. The cloned
+  /// evaluators are then used to evaluate the functions. If this is a non-grouping
+  /// aggregation these evaluators are used directly to evaluate the functions.
+  ///
+  /// Permanent and result allocations for these allocators are allocated from
+  /// 'expr_perm_pool_' and 'expr_results_pool_' respectively.
+  std::vector<AggFnEvaluator*> agg_fn_evals_;
+
+  /// Conjuncts and their evaluators in this aggregator. 'conjuncts_' live in the
+  /// query-state's object pool while the evaluators live in this aggregator's
+  /// object pool.
+  std::vector<ScalarExpr*> conjuncts_;
+  std::vector<ScalarExprEvaluator*> conjunct_evals_;
+
+  /// Runtime profile for this aggregator. Owned by 'pool_'.
+  RuntimeProfile* const runtime_profile_;
+
+  int64_t num_rows_returned_;
+  RuntimeProfile::Counter* rows_returned_counter_;
+
+  /// Time spent processing the child rows
+  RuntimeProfile::Counter* build_timer_;
+
+  /// Initializes the aggregate function slots of an intermediate tuple.
+  /// Any var-len data is allocated from the FunctionContexts.
+  void InitAggSlots(
+      const std::vector<AggFnEvaluator*>& agg_fn_evals, Tuple* intermediate_tuple);
+
+  /// Updates the given aggregation intermediate tuple with aggregation values computed
+  /// over 'row' using 'agg_fn_evals'. Whether the agg fn evaluator calls Update() or
+  /// Merge() is controlled by the evaluator itself, unless enforced explicitly by passing
+  /// in is_merge == true.  The override is needed to merge spilled and non-spilled rows
+  /// belonging to the same partition independent of whether the agg fn evaluators have
+  /// is_merge() == true.
+  /// This function is replaced by codegen (which is why we don't use a vector argument
+  /// for agg_fn_evals).. Any var-len data is allocated from the FunctionContexts.
+  /// TODO: Fix the arguments order. Need to update CodegenUpdateTuple() too.
+  void UpdateTuple(AggFnEvaluator** agg_fn_evals, Tuple* tuple, TupleRow* row,
+      bool is_merge = false) noexcept;
+
+  /// Called on the intermediate tuple of each group after all input rows have been
+  /// consumed and aggregated. Computes the final aggregate values to be returned in
+  /// GetNext() using the agg fn evaluators' Serialize() or Finalize().
+  /// For the Finalize() case if the output tuple is different from the intermediate
+  /// tuple, then a new tuple is allocated from 'pool' to hold the final result.
+  /// Grouping values are copied into the output tuple and the the output tuple holding
+  /// the finalized/serialized aggregate values is returned.
+  /// TODO: Coordinate the allocation of new tuples with the release of memory
+  /// so as not to make memory consumption blow up.
+  Tuple* GetOutputTuple(
+      const std::vector<AggFnEvaluator*>& agg_fn_evals, Tuple* tuple, MemPool* pool);
+
+  /// Codegen for updating aggregate expressions agg_fns_[agg_fn_idx]
+  /// and returns the IR function in 'fn'. Returns non-OK status if codegen
+  /// is unsuccessful.
+  Status CodegenUpdateSlot(LlvmCodeGen* codegen, int agg_fn_idx,
+      SlotDescriptor* slot_desc, llvm::Function** fn) WARN_UNUSED_RESULT;
+
+  /// Codegen a call to a function implementing the UDA interface with input values
+  /// from 'input_vals'. 'dst_val' should contain the previous value of the aggregate
+  /// function, and 'updated_dst_val' is set to the new value after the Update or Merge
+  /// operation is applied. The instruction sequence for the UDA call is inserted at
+  /// the insert position of 'builder'.
+  Status CodegenCallUda(LlvmCodeGen* codegen, LlvmBuilder* builder, AggFn* agg_fn,
+      llvm::Value* agg_fn_ctx_arg, const std::vector<CodegenAnyVal>& input_vals,
+      const CodegenAnyVal& dst_val, CodegenAnyVal* updated_dst_val) WARN_UNUSED_RESULT;
+
+  /// Codegen UpdateTuple(). Returns non-OK status if codegen is unsuccessful.
+  Status CodegenUpdateTuple(LlvmCodeGen* codegen, llvm::Function** fn) WARN_UNUSED_RESULT;
+};
+} // namespace impala
+
+#endif // IMPALA_EXEC_AGGREGATOR_H
diff --git a/be/src/exec/exec-node.cc b/be/src/exec/exec-node.cc
index 7cdd054..a44a5c1 100644
--- a/be/src/exec/exec-node.cc
+++ b/be/src/exec/exec-node.cc
@@ -29,6 +29,7 @@
 #include "common/status.h"
 #include "exprs/scalar-expr.h"
 #include "exprs/scalar-expr-evaluator.h"
+#include "exec/aggregation-node.h"
 #include "exec/analytic-eval-node.h"
 #include "exec/cardinality-check-node.h"
 #include "exec/data-source-scan-node.h"
@@ -42,11 +43,11 @@
 #include "exec/kudu-util.h"
 #include "exec/nested-loop-join-node.h"
 #include "exec/partial-sort-node.h"
-#include "exec/partitioned-aggregation-node.h"
 #include "exec/partitioned-hash-join-node.h"
 #include "exec/select-node.h"
 #include "exec/singular-row-src-node.h"
 #include "exec/sort-node.h"
+#include "exec/streaming-aggregation-node.h"
 #include "exec/subplan-node.h"
 #include "exec/topn-node.h"
 #include "exec/union-node.h"
@@ -306,7 +307,11 @@ Status ExecNode::CreateNode(ObjectPool* pool, const TPlanNode& tnode,
       }
       break;
     case TPlanNodeType::AGGREGATION_NODE:
-      *node = pool->Add(new PartitionedAggregationNode(pool, tnode, descs));
+      if (tnode.agg_node.use_streaming_preaggregation) {
+        *node = pool->Add(new StreamingAggregationNode(pool, tnode, descs));
+      } else {
+        *node = pool->Add(new AggregationNode(pool, tnode, descs));
+      }
       break;
     case TPlanNodeType::HASH_JOIN_NODE:
       *node = pool->Add(new PartitionedHashJoinNode(pool, tnode, descs));
diff --git a/be/src/exec/exec-node.h b/be/src/exec/exec-node.h
index ad9ae10..9a87a56 100644
--- a/be/src/exec/exec-node.h
+++ b/be/src/exec/exec-node.h
@@ -211,6 +211,7 @@ class ExecNode {
   MemTracker* expr_mem_tracker() { return expr_mem_tracker_.get(); }
   MemPool* expr_perm_pool() { return expr_perm_pool_.get(); }
   MemPool* expr_results_pool() { return expr_results_pool_.get(); }
+  const TBackendResourceProfile& resource_profile() { return resource_profile_; }
   bool is_closed() const { return is_closed_; }
 
   /// Return true if codegen was disabled by the planner for this ExecNode. Does not
@@ -220,6 +221,10 @@ class ExecNode {
   /// Extract node id from p->name().
   static int GetNodeIdFromProfile(RuntimeProfile* p);
 
+  /// Returns true if this node is inside the right-hand side plan tree of a SubplanNode.
+  /// Valid to call in or after Prepare().
+  bool IsInSubplan() const { return containing_subplan_ != NULL; }
+
   /// Names of counters shared by all exec nodes
   static const std::string ROW_THROUGHPUT_COUNTER;
 
@@ -322,10 +327,6 @@ class ExecNode {
   /// Set by SubplanNode::Init(). Not owned.
   SubplanNode* containing_subplan_;
 
-  /// Returns true if this node is inside the right-hand side plan tree of a SubplanNode.
-  /// Valid to call in or after Prepare().
-  bool IsInSubplan() const { return containing_subplan_ != NULL; }
-
   /// If true, codegen should be disabled for this exec node.
   const bool disable_codegen_;
 
diff --git a/be/src/exec/partitioned-aggregation-node-ir.cc b/be/src/exec/grouping-aggregator-ir.cc
similarity index 73%
rename from be/src/exec/partitioned-aggregation-node-ir.cc
rename to be/src/exec/grouping-aggregator-ir.cc
index 69d297c..d3dbf17 100644
--- a/be/src/exec/partitioned-aggregation-node-ir.cc
+++ b/be/src/exec/grouping-aggregator-ir.cc
@@ -15,28 +15,17 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "exec/partitioned-aggregation-node.h"
+#include "exec/grouping-aggregator.h"
 
 #include "exec/hash-table.inline.h"
 #include "exprs/agg-fn-evaluator.h"
-#include "exprs/scalar-expr.h"
-#include "exprs/scalar-expr-evaluator.h"
-#include "runtime/buffered-tuple-stream.inline.h"
 #include "runtime/row-batch.h"
 #include "runtime/tuple-row.h"
 
 using namespace impala;
 
-Status PartitionedAggregationNode::ProcessBatchNoGrouping(RowBatch* batch) {
-  Tuple* output_tuple = singleton_output_tuple_;
-  FOREACH_ROW(batch, 0, batch_iter) {
-    UpdateTuple(agg_fn_evals_.data(), output_tuple, batch_iter.Get());
-  }
-  return Status::OK();
-}
-
-template<bool AGGREGATED_ROWS>
-Status PartitionedAggregationNode::ProcessBatch(RowBatch* batch,
+template <bool AGGREGATED_ROWS>
+Status GroupingAggregator::AddBatchImpl(RowBatch* batch,
     TPrefetchMode::type prefetch_mode, HashTableCtx* __restrict__ ht_ctx) {
   DCHECK(!hash_partitions_.empty());
   DCHECK(!is_streaming_preagg_);
@@ -64,10 +53,9 @@ Status PartitionedAggregationNode::ProcessBatch(RowBatch* batch,
   return Status::OK();
 }
 
-template<bool AGGREGATED_ROWS>
-void IR_ALWAYS_INLINE PartitionedAggregationNode::EvalAndHashPrefetchGroup(
-    RowBatch* batch, int start_row_idx, TPrefetchMode::type prefetch_mode,
-    HashTableCtx* ht_ctx) {
+template <bool AGGREGATED_ROWS>
+void IR_ALWAYS_INLINE GroupingAggregator::EvalAndHashPrefetchGroup(RowBatch* batch,
+    int start_row_idx, TPrefetchMode::type prefetch_mode, HashTableCtx* ht_ctx) {
   HashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache();
   const int cache_size = expr_vals_cache->capacity();
 
@@ -87,7 +75,7 @@ void IR_ALWAYS_INLINE PartitionedAggregationNode::EvalAndHashPrefetchGroup(
     if (is_null) {
       expr_vals_cache->SetRowNull();
     } else if (prefetch_mode != TPrefetchMode::NONE) {
-      if (LIKELY(hash_tbl != NULL)) hash_tbl->PrefetchBucket<false>(hash);
+      if (LIKELY(hash_tbl != nullptr)) hash_tbl->PrefetchBucket<false>(hash);
     }
     expr_vals_cache->NextRow();
   }
@@ -95,9 +83,9 @@ void IR_ALWAYS_INLINE PartitionedAggregationNode::EvalAndHashPrefetchGroup(
   expr_vals_cache->ResetForRead();
 }
 
-template<bool AGGREGATED_ROWS>
-Status PartitionedAggregationNode::ProcessRow(TupleRow* __restrict__ row,
-    HashTableCtx* __restrict__ ht_ctx) {
+template <bool AGGREGATED_ROWS>
+Status GroupingAggregator::ProcessRow(
+    TupleRow* __restrict__ row, HashTableCtx* __restrict__ ht_ctx) {
   HashTableCtx::ExprValuesCache* expr_vals_cache = ht_ctx->expr_values_cache();
   // Hoist lookups out of non-null branch to speed up non-null case.
   const uint32_t hash = expr_vals_cache->CurExprValuesHash();
@@ -110,8 +98,8 @@ Status PartitionedAggregationNode::ProcessRow(TupleRow* __restrict__ row,
   HashTable* hash_tbl = GetHashTable(partition_idx);
   Partition* dst_partition = hash_partitions_[partition_idx];
   DCHECK(dst_partition != nullptr);
-  DCHECK_EQ(dst_partition->is_spilled(), hash_tbl == NULL);
-  if (hash_tbl == NULL) {
+  DCHECK_EQ(dst_partition->is_spilled(), hash_tbl == nullptr);
+  if (hash_tbl == nullptr) {
     // This partition is already spilled, just append the row.
     return AppendSpilledRow<AGGREGATED_ROWS>(dst_partition, row);
   }
@@ -138,22 +126,22 @@ Status PartitionedAggregationNode::ProcessRow(TupleRow* __restrict__ row,
   return AddIntermediateTuple<AGGREGATED_ROWS>(dst_partition, row, hash, it);
 }
 
-template<bool AGGREGATED_ROWS>
-Status PartitionedAggregationNode::AddIntermediateTuple(Partition* __restrict__ partition,
+template <bool AGGREGATED_ROWS>
+Status GroupingAggregator::AddIntermediateTuple(Partition* __restrict__ partition,
     TupleRow* __restrict__ row, uint32_t hash, HashTable::Iterator insert_it) {
   while (true) {
     DCHECK(partition->aggregated_row_stream->is_pinned());
     Tuple* intermediate_tuple = ConstructIntermediateTuple(partition->agg_fn_evals,
-        partition->aggregated_row_stream.get(), &process_batch_status_);
+        partition->aggregated_row_stream.get(), &add_batch_status_);
 
-    if (LIKELY(intermediate_tuple != NULL)) {
-      UpdateTuple(partition->agg_fn_evals.data(), intermediate_tuple,
-          row, AGGREGATED_ROWS);
+    if (LIKELY(intermediate_tuple != nullptr)) {
+      UpdateTuple(
+          partition->agg_fn_evals.data(), intermediate_tuple, row, AGGREGATED_ROWS);
       // After copying and initializing the tuple, insert it into the hash table.
       insert_it.SetTuple(intermediate_tuple, hash);
       return Status::OK();
-    } else if (!process_batch_status_.ok()) {
-      return std::move(process_batch_status_);
+    } else if (!add_batch_status_.ok()) {
+      return std::move(add_batch_status_);
     }
 
     // We did not have enough memory to add intermediate_tuple to the stream.
@@ -164,7 +152,7 @@ Status PartitionedAggregationNode::AddIntermediateTuple(Partition* __restrict__
   }
 }
 
-Status PartitionedAggregationNode::ProcessBatchStreaming(bool needs_serialize,
+Status GroupingAggregator::AddBatchStreamingImpl(bool needs_serialize,
     TPrefetchMode::type prefetch_mode, RowBatch* in_batch, RowBatch* out_batch,
     HashTableCtx* __restrict__ ht_ctx, int remaining_capacity[PARTITION_FANOUT]) {
   DCHECK(is_streaming_preagg_);
@@ -183,24 +171,24 @@ Status PartitionedAggregationNode::ProcessBatchStreaming(bool needs_serialize,
       TupleRow* in_row = in_batch_iter.Get();
       const uint32_t hash = expr_vals_cache->CurExprValuesHash();
       const uint32_t partition_idx = hash >> (32 - NUM_PARTITIONING_BITS);
-      if (!expr_vals_cache->IsRowNull() &&
-          !TryAddToHashTable(ht_ctx, hash_partitions_[partition_idx],
-            GetHashTable(partition_idx), in_row, hash, &remaining_capacity[partition_idx],
-            &process_batch_status_)) {
-        RETURN_IF_ERROR(std::move(process_batch_status_));
+      if (!expr_vals_cache->IsRowNull()
+          && !TryAddToHashTable(ht_ctx, hash_partitions_[partition_idx],
+                 GetHashTable(partition_idx), in_row, hash,
+                 &remaining_capacity[partition_idx], &add_batch_status_)) {
+        RETURN_IF_ERROR(std::move(add_batch_status_));
         // Tuple is not going into hash table, add it to the output batch.
-        Tuple* intermediate_tuple = ConstructIntermediateTuple(agg_fn_evals_,
-            out_batch->tuple_data_pool(), &process_batch_status_);
-        if (UNLIKELY(intermediate_tuple == NULL)) {
-          DCHECK(!process_batch_status_.ok());
-          return std::move(process_batch_status_);
+        Tuple* intermediate_tuple = ConstructIntermediateTuple(
+            agg_fn_evals_, out_batch->tuple_data_pool(), &add_batch_status_);
+        if (UNLIKELY(intermediate_tuple == nullptr)) {
+          DCHECK(!add_batch_status_.ok());
+          return std::move(add_batch_status_);
         }
         UpdateTuple(agg_fn_evals_.data(), intermediate_tuple, in_row);
         out_batch_iterator.Get()->SetTuple(0, intermediate_tuple);
         out_batch_iterator.Next();
         out_batch->CommitLastRow();
       }
-      DCHECK(process_batch_status_.ok());
+      DCHECK(add_batch_status_.ok());
       expr_vals_cache->NextRow();
     }
     DCHECK(expr_vals_cache->AtEnd());
@@ -214,11 +202,11 @@ Status PartitionedAggregationNode::ProcessBatchStreaming(bool needs_serialize,
   return Status::OK();
 }
 
-bool PartitionedAggregationNode::TryAddToHashTable(
-    HashTableCtx* __restrict__ ht_ctx, Partition* __restrict__ partition,
-    HashTable* __restrict__ hash_tbl, TupleRow* __restrict__ in_row,
-    uint32_t hash, int* __restrict__ remaining_capacity, Status* status) {
-  DCHECK(remaining_capacity != NULL);
+bool GroupingAggregator::TryAddToHashTable(HashTableCtx* __restrict__ ht_ctx,
+    Partition* __restrict__ partition, HashTable* __restrict__ hash_tbl,
+    TupleRow* __restrict__ in_row, uint32_t hash, int* __restrict__ remaining_capacity,
+    Status* status) {
+  DCHECK(remaining_capacity != nullptr);
   DCHECK_EQ(hash_tbl, partition->hash_tbl.get());
   DCHECK_GE(*remaining_capacity, 0);
   bool found;
@@ -230,9 +218,9 @@ bool PartitionedAggregationNode::TryAddToHashTable(
   } else if (*remaining_capacity == 0) {
     return false;
   } else {
-    intermediate_tuple = ConstructIntermediateTuple(partition->agg_fn_evals,
-        partition->aggregated_row_stream.get(), status);
-    if (LIKELY(intermediate_tuple != NULL)) {
+    intermediate_tuple = ConstructIntermediateTuple(
+        partition->agg_fn_evals, partition->aggregated_row_stream.get(), status);
+    if (LIKELY(intermediate_tuple != nullptr)) {
       it.SetTuple(intermediate_tuple, hash);
       --(*remaining_capacity);
     } else {
@@ -247,7 +235,7 @@ bool PartitionedAggregationNode::TryAddToHashTable(
 }
 
 // Instantiate required templates.
-template Status PartitionedAggregationNode::ProcessBatch<false>(RowBatch*,
-    TPrefetchMode::type, HashTableCtx*);
-template Status PartitionedAggregationNode::ProcessBatch<true>(RowBatch*,
-    TPrefetchMode::type, HashTableCtx*);
+template Status GroupingAggregator::AddBatchImpl<false>(
+    RowBatch*, TPrefetchMode::type, HashTableCtx*);
+template Status GroupingAggregator::AddBatchImpl<true>(
+    RowBatch*, TPrefetchMode::type, HashTableCtx*);
diff --git a/be/src/exec/grouping-aggregator-partition.cc b/be/src/exec/grouping-aggregator-partition.cc
new file mode 100644
index 0000000..8fe08f4
--- /dev/null
+++ b/be/src/exec/grouping-aggregator-partition.cc
@@ -0,0 +1,218 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/grouping-aggregator.h"
+
+#include <set>
+#include <sstream>
+
+#include "exec/exec-node.h"
+#include "exec/hash-table.inline.h"
+#include "exprs/agg-fn-evaluator.h"
+#include "runtime/descriptors.h"
+#include "runtime/mem-pool.h"
+#include "runtime/row-batch.h"
+#include "runtime/runtime-state.h"
+#include "util/runtime-profile-counters.h"
+
+#include "gen-cpp/PlanNodes_types.h"
+
+namespace impala {
+
+GroupingAggregator::Partition::~Partition() {
+  DCHECK(is_closed);
+}
+
+Status GroupingAggregator::Partition::InitStreams() {
+  agg_fn_perm_pool.reset(new MemPool(parent->expr_mem_tracker_.get()));
+  DCHECK_EQ(agg_fn_evals.size(), 0);
+  AggFnEvaluator::ShallowClone(parent->partition_pool_.get(), agg_fn_perm_pool.get(),
+      parent->expr_results_pool_.get(), parent->agg_fn_evals_, &agg_fn_evals);
+  // Varlen aggregate function results are stored outside of aggregated_row_stream because
+  // BufferedTupleStream doesn't support relocating varlen data stored in the stream.
+  auto agg_slot =
+      parent->intermediate_tuple_desc_->slots().begin() + parent->grouping_exprs_.size();
+  std::set<SlotId> external_varlen_slots;
+  for (; agg_slot != parent->intermediate_tuple_desc_->slots().end(); ++agg_slot) {
+    if ((*agg_slot)->type().IsVarLenStringType()) {
+      external_varlen_slots.insert((*agg_slot)->id());
+    }
+  }
+
+  aggregated_row_stream.reset(
+      new BufferedTupleStream(parent->state_, &parent->intermediate_row_desc_,
+          parent->buffer_pool_client(), parent->resource_profile_.spillable_buffer_size,
+          parent->resource_profile_.max_row_buffer_size, external_varlen_slots));
+  RETURN_IF_ERROR(aggregated_row_stream->Init(parent->id_, true));
+  bool got_buffer;
+  RETURN_IF_ERROR(aggregated_row_stream->PrepareForWrite(&got_buffer));
+  DCHECK(got_buffer) << "Buffer included in reservation " << parent->id_ << "\n"
+                     << parent->buffer_pool_client()->DebugString() << "\n"
+                     << parent->DebugString(2);
+  if (!parent->is_streaming_preagg_) {
+    unaggregated_row_stream.reset(
+        new BufferedTupleStream(parent->state_, &parent->input_row_desc_,
+            parent->buffer_pool_client(), parent->resource_profile_.spillable_buffer_size,
+            parent->resource_profile_.max_row_buffer_size));
+    // This stream is only used to spill, no need to ever have this pinned.
+    RETURN_IF_ERROR(unaggregated_row_stream->Init(parent->id_, false));
+    // Save memory by waiting until we spill to allocate the write buffer for the
+    // unaggregated row stream.
+    DCHECK(!unaggregated_row_stream->has_write_iterator());
+  }
+  return Status::OK();
+}
+
+Status GroupingAggregator::Partition::InitHashTable(bool* got_memory) {
+  DCHECK(aggregated_row_stream != nullptr);
+  DCHECK(hash_tbl == nullptr);
+  // We use the upper PARTITION_FANOUT num bits to pick the partition so only the
+  // remaining bits can be used for the hash table.
+  // TODO: we could switch to 64 bit hashes and then we don't need a max size.
+  // It might be reasonable to limit individual hash table size for other reasons
+  // though. Always start with small buffers.
+  hash_tbl.reset(HashTable::Create(parent->ht_allocator_.get(), false, 1, nullptr,
+      1L << (32 - NUM_PARTITIONING_BITS), PAGG_DEFAULT_HASH_TABLE_SZ));
+  // Please update the error message in CreateHashPartitions() if initial size of
+  // hash table changes.
+  return hash_tbl->Init(got_memory);
+}
+
+Status GroupingAggregator::Partition::SerializeStreamForSpilling() {
+  DCHECK(!parent->is_streaming_preagg_);
+  if (parent->needs_serialize_) {
+    // We need to do a lot more work in this case. This step effectively does a merge
+    // aggregation in this node. We need to serialize the intermediates, spill the
+    // intermediates and then feed them into the aggregate function's merge step.
+    // This is often used when the intermediate is a string type, meaning the current
+    // (before serialization) in-memory layout is not the on-disk block layout.
+    // The disk layout does not support mutable rows. We need to rewrite the stream
+    // into the on disk format.
+    // TODO: if it happens to not be a string, we could serialize in place. This is
+    // a future optimization since it is very unlikely to have a serialize phase
+    // for those UDAs.
+    DCHECK(parent->serialize_stream_.get() != nullptr);
+    DCHECK(!parent->serialize_stream_->is_pinned());
+
+    // Serialize and copy the spilled partition's stream into the new stream.
+    Status status;
+    BufferedTupleStream* new_stream = parent->serialize_stream_.get();
+    HashTable::Iterator it = hash_tbl->Begin(parent->ht_ctx_.get());
+    while (!it.AtEnd()) {
+      Tuple* tuple = it.GetTuple();
+      it.Next();
+      AggFnEvaluator::Serialize(agg_fn_evals, tuple);
+      if (UNLIKELY(!new_stream->AddRow(reinterpret_cast<TupleRow*>(&tuple), &status))) {
+        DCHECK(!status.ok()) << "Stream was unpinned - AddRow() only fails on error";
+        // Even if we can't add to new_stream, finish up processing this agg stream to
+        // make clean up easier (someone has to finalize this stream and we don't want to
+        // remember where we are).
+        parent->CleanupHashTbl(agg_fn_evals, it);
+        hash_tbl->Close();
+        hash_tbl.reset();
+        aggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+        return status;
+      }
+    }
+
+    aggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+    aggregated_row_stream.swap(parent->serialize_stream_);
+    // Recreate the serialize_stream (and reserve 1 buffer) now in preparation for
+    // when we need to spill again. We need to have this available before we need
+    // to spill to make sure it is available. This should be acquirable since we just
+    // freed at least one buffer from this partition's (old) aggregated_row_stream.
+    parent->serialize_stream_.reset(
+        new BufferedTupleStream(parent->state_, &parent->intermediate_row_desc_,
+            parent->buffer_pool_client(), parent->resource_profile_.spillable_buffer_size,
+            parent->resource_profile_.max_row_buffer_size));
+    status = parent->serialize_stream_->Init(parent->id_, false);
+    if (status.ok()) {
+      bool got_buffer;
+      status = parent->serialize_stream_->PrepareForWrite(&got_buffer);
+      DCHECK(!status.ok() || got_buffer) << "Accounted in min reservation";
+    }
+    if (!status.ok()) {
+      hash_tbl->Close();
+      hash_tbl.reset();
+      return status;
+    }
+    DCHECK(parent->serialize_stream_->has_write_iterator());
+  }
+  return Status::OK();
+}
+
+Status GroupingAggregator::Partition::Spill(bool more_aggregate_rows) {
+  DCHECK(!parent->is_streaming_preagg_);
+  DCHECK(!is_closed);
+  DCHECK(!is_spilled());
+  RETURN_IF_ERROR(parent->state_->StartSpilling(parent->mem_tracker_.get()));
+
+  RETURN_IF_ERROR(SerializeStreamForSpilling());
+
+  // Free the in-memory result data.
+  AggFnEvaluator::Close(agg_fn_evals, parent->state_);
+  agg_fn_evals.clear();
+
+  if (agg_fn_perm_pool.get() != nullptr) {
+    agg_fn_perm_pool->FreeAll();
+    agg_fn_perm_pool.reset();
+  }
+
+  hash_tbl->Close();
+  hash_tbl.reset();
+
+  // Unpin the stream to free memory, but leave a write buffer in place so we can
+  // continue appending rows to one of the streams in the partition.
+  DCHECK(aggregated_row_stream->has_write_iterator());
+  DCHECK(!unaggregated_row_stream->has_write_iterator());
+  if (more_aggregate_rows) {
+    aggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL_EXCEPT_CURRENT);
+  } else {
+    aggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL);
+    bool got_buffer;
+    RETURN_IF_ERROR(unaggregated_row_stream->PrepareForWrite(&got_buffer));
+    DCHECK(got_buffer) << "Accounted in min reservation"
+                       << parent->buffer_pool_client()->DebugString();
+  }
+
+  COUNTER_ADD(parent->num_spilled_partitions_, 1);
+  if (parent->num_spilled_partitions_->value() == 1) {
+    parent->runtime_profile()->AppendExecOption("Spilled");
+  }
+  return Status::OK();
+}
+
+void GroupingAggregator::Partition::Close(bool finalize_rows) {
+  if (is_closed) return;
+  is_closed = true;
+  if (aggregated_row_stream.get() != nullptr) {
+    if (finalize_rows && hash_tbl.get() != nullptr) {
+      // We need to walk all the rows and Finalize them here so the UDA gets a chance
+      // to cleanup. If the hash table is gone (meaning this was spilled), the rows
+      // should have been finalized/serialized in Spill().
+      parent->CleanupHashTbl(agg_fn_evals, hash_tbl->Begin(parent->ht_ctx_.get()));
+    }
+    aggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+  }
+  if (hash_tbl.get() != nullptr) hash_tbl->Close();
+  if (unaggregated_row_stream.get() != nullptr) {
+    unaggregated_row_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+  }
+  for (AggFnEvaluator* eval : agg_fn_evals) eval->Close(parent->state_);
+  if (agg_fn_perm_pool.get() != nullptr) agg_fn_perm_pool->FreeAll();
+}
+} // namespace impala
diff --git a/be/src/exec/grouping-aggregator.cc b/be/src/exec/grouping-aggregator.cc
new file mode 100644
index 0000000..60001a6
--- /dev/null
+++ b/be/src/exec/grouping-aggregator.cc
@@ -0,0 +1,1098 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/grouping-aggregator.h"
+
+#include <sstream>
+
+#include "codegen/llvm-codegen.h"
+#include "exec/exec-node.h"
+#include "exec/hash-table.inline.h"
+#include "exprs/agg-fn-evaluator.h"
+#include "exprs/scalar-expr.h"
+#include "exprs/slot-ref.h"
+#include "gutil/strings/substitute.h"
+#include "runtime/buffered-tuple-stream.inline.h"
+#include "runtime/descriptors.h"
+#include "runtime/exec-env.h"
+#include "runtime/mem-pool.h"
+#include "runtime/mem-tracker.h"
+#include "runtime/row-batch.h"
+#include "runtime/runtime-state.h"
+#include "runtime/tuple-row.h"
+#include "runtime/tuple.h"
+#include "util/runtime-profile-counters.h"
+
+#include "gen-cpp/PlanNodes_types.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+/// The minimum reduction factor (input rows divided by output rows) to grow hash tables
+/// in a streaming preaggregation, given that the hash tables are currently the given
+/// size or above. The sizes roughly correspond to hash table sizes where the bucket
+/// arrays will fit in  a cache level. Intuitively, we don't want the working set of the
+/// aggregation to expand to the next level of cache unless we're reducing the input
+/// enough to outweigh the increased memory latency we'll incur for each hash table
+/// lookup.
+///
+/// Note that the current reduction achieved is not always a good estimate of the
+/// final reduction. It may be biased either way depending on the ordering of the
+/// input. If the input order is random, we will underestimate the final reduction
+/// factor because the probability of a row having the same key as a previous row
+/// increases as more input is processed.  If the input order is correlated with the
+/// key, skew may bias the estimate. If high cardinality keys appear first, we
+/// may overestimate and if low cardinality keys appear first, we underestimate.
+/// To estimate the eventual reduction achieved, we estimate the final reduction
+/// using the planner's estimated input cardinality and the assumption that input
+/// is in a random order. This means that we assume that the reduction factor will
+/// increase over time.
+struct StreamingHtMinReductionEntry {
+  // Use 'streaming_ht_min_reduction' if the total size of hash table bucket directories
+  // in bytes is greater than this threshold.
+  int min_ht_mem;
+  // The minimum reduction factor to expand the hash tables.
+  double streaming_ht_min_reduction;
+};
+
+// TODO: experimentally tune these values and also programmatically get the cache size
+// of the machine that we're running on.
+static const StreamingHtMinReductionEntry STREAMING_HT_MIN_REDUCTION[] = {
+    // Expand up to L2 cache always.
+    {0, 0.0},
+    // Expand into L3 cache if we look like we're getting some reduction.
+    {256 * 1024, 1.1},
+    // Expand into main memory if we're getting a significant reduction.
+    {2 * 1024 * 1024, 2.0},
+};
+
+static const int STREAMING_HT_MIN_REDUCTION_SIZE =
+    sizeof(STREAMING_HT_MIN_REDUCTION) / sizeof(STREAMING_HT_MIN_REDUCTION[0]);
+
+GroupingAggregator::GroupingAggregator(ExecNode* exec_node, ObjectPool* pool,
+    const TPlanNode& tnode, const DescriptorTbl& descs)
+  : Aggregator(exec_node, pool, tnode, descs, "GroupingAggregator"),
+    intermediate_row_desc_(intermediate_tuple_desc_, false),
+    is_streaming_preagg_(tnode.agg_node.use_streaming_preaggregation),
+    needs_serialize_(false),
+    output_partition_(nullptr),
+    resource_profile_(exec_node->resource_profile()),
+    num_input_rows_(0),
+    is_in_subplan_(exec_node->IsInSubplan()),
+    limit_(exec_node->limit()),
+    add_batch_impl_fn_(nullptr),
+    add_batch_streaming_impl_fn_(nullptr),
+    ht_resize_timer_(nullptr),
+    get_results_timer_(nullptr),
+    num_hash_buckets_(nullptr),
+    partitions_created_(nullptr),
+    max_partition_level_(nullptr),
+    num_row_repartitioned_(nullptr),
+    num_repartitions_(nullptr),
+    num_spilled_partitions_(nullptr),
+    largest_partition_percent_(nullptr),
+    streaming_timer_(nullptr),
+    num_passthrough_rows_(nullptr),
+    preagg_estimated_reduction_(nullptr),
+    preagg_streaming_ht_min_reduction_(nullptr),
+    estimated_input_cardinality_(tnode.agg_node.estimated_input_cardinality),
+    partition_eos_(false),
+    partition_pool_(new ObjectPool()) {
+  DCHECK_EQ(PARTITION_FANOUT, 1 << NUM_PARTITIONING_BITS);
+}
+
+Status GroupingAggregator::Init(const TPlanNode& tnode, RuntimeState* state) {
+  RETURN_IF_ERROR(ScalarExpr::Create(
+      tnode.agg_node.grouping_exprs, input_row_desc_, state, &grouping_exprs_));
+
+  // Construct build exprs from intermediate_row_desc_
+  for (int i = 0; i < grouping_exprs_.size(); ++i) {
+    SlotDescriptor* desc = intermediate_tuple_desc_->slots()[i];
+    DCHECK(desc->type().type == TYPE_NULL || desc->type() == grouping_exprs_[i]->type());
+    // Hack to avoid TYPE_NULL SlotRefs.
+    SlotRef* build_expr =
+        pool_->Add(desc->type().type != TYPE_NULL ? new SlotRef(desc) :
+                                                    new SlotRef(desc, TYPE_BOOLEAN));
+    build_exprs_.push_back(build_expr);
+    RETURN_IF_ERROR(build_expr->Init(intermediate_row_desc_, state));
+    if (build_expr->type().IsVarLenStringType()) string_grouping_exprs_.push_back(i);
+  }
+
+  RETURN_IF_ERROR(Aggregator::Init(tnode, state));
+  for (int i = 0; i < agg_fns_.size(); ++i) {
+    needs_serialize_ |= agg_fns_[i]->SupportsSerialize();
+  }
+  return Status::OK();
+}
+
+Status GroupingAggregator::Prepare(RuntimeState* state) {
+  RETURN_IF_ERROR(Aggregator::Prepare(state));
+  state_ = state;
+  tuple_pool_.reset(new MemPool(mem_tracker_.get()));
+
+  ht_resize_timer_ = ADD_TIMER(runtime_profile(), "HTResizeTime");
+  get_results_timer_ = ADD_TIMER(runtime_profile(), "GetResultsTime");
+  num_hash_buckets_ = ADD_COUNTER(runtime_profile(), "HashBuckets", TUnit::UNIT);
+  partitions_created_ = ADD_COUNTER(runtime_profile(), "PartitionsCreated", TUnit::UNIT);
+  largest_partition_percent_ =
+      runtime_profile()->AddHighWaterMarkCounter("LargestPartitionPercent", TUnit::UNIT);
+  if (is_streaming_preagg_) {
+    runtime_profile()->AppendExecOption("Streaming Preaggregation");
+    streaming_timer_ = ADD_TIMER(runtime_profile(), "StreamingTime");
+    num_passthrough_rows_ =
+        ADD_COUNTER(runtime_profile(), "RowsPassedThrough", TUnit::UNIT);
+    preagg_estimated_reduction_ =
+        ADD_COUNTER(runtime_profile(), "ReductionFactorEstimate", TUnit::DOUBLE_VALUE);
+    preagg_streaming_ht_min_reduction_ = ADD_COUNTER(
+        runtime_profile(), "ReductionFactorThresholdToExpand", TUnit::DOUBLE_VALUE);
+  } else {
+    num_row_repartitioned_ =
+        ADD_COUNTER(runtime_profile(), "RowsRepartitioned", TUnit::UNIT);
+    num_repartitions_ = ADD_COUNTER(runtime_profile(), "NumRepartitions", TUnit::UNIT);
+    num_spilled_partitions_ =
+        ADD_COUNTER(runtime_profile(), "SpilledPartitions", TUnit::UNIT);
+    max_partition_level_ =
+        runtime_profile()->AddHighWaterMarkCounter("MaxPartitionLevel", TUnit::UNIT);
+  }
+
+  RETURN_IF_ERROR(HashTableCtx::Create(pool_, state, build_exprs_, grouping_exprs_, true,
+      vector<bool>(build_exprs_.size(), true), state->fragment_hash_seed(),
+      MAX_PARTITION_DEPTH, 1, expr_perm_pool_.get(), expr_results_pool_.get(),
+      expr_results_pool_.get(), &ht_ctx_));
+
+  reservation_manager_.Init(
+      Substitute("GroupingAggregator id=$0 ptr=$1", id_, this), runtime_profile_,
+      mem_tracker_.get(), resource_profile_, debug_options_);
+  return Status::OK();
+}
+
+void GroupingAggregator::Codegen(RuntimeState* state) {
+  LlvmCodeGen* codegen = state->codegen();
+  DCHECK(codegen != nullptr);
+  TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
+  Status codegen_status = is_streaming_preagg_ ?
+      CodegenAddBatchStreamingImpl(codegen, prefetch_mode) :
+      CodegenAddBatchImpl(codegen, prefetch_mode);
+  runtime_profile()->AddCodegenMsg(codegen_status.ok(), codegen_status);
+}
+
+Status GroupingAggregator::Open(RuntimeState* state) {
+  RETURN_IF_ERROR(Aggregator::Open(state));
+
+  // Claim reservation after the child has been opened to reduce the peak reservation
+  // requirement.
+  if (!buffer_pool_client()->is_registered()) {
+    DCHECK_GE(resource_profile_.min_reservation, MinReservation());
+    RETURN_IF_ERROR(reservation_manager_.ClaimBufferReservation(state));
+  }
+
+  DCHECK(ht_ctx_.get() != nullptr);
+  RETURN_IF_ERROR(ht_ctx_->Open(state));
+
+  if (ht_allocator_ == nullptr) {
+    // Allocate 'serialize_stream_' and 'ht_allocator_' on the first Open() call.
+    ht_allocator_.reset(new Suballocator(state_->exec_env()->buffer_pool(),
+        buffer_pool_client(), resource_profile_.spillable_buffer_size));
+
+    if (!is_streaming_preagg_ && needs_serialize_) {
+      serialize_stream_.reset(new BufferedTupleStream(state, &intermediate_row_desc_,
+          buffer_pool_client(), resource_profile_.spillable_buffer_size,
+          resource_profile_.max_row_buffer_size));
+      RETURN_IF_ERROR(serialize_stream_->Init(id_, false));
+      bool got_buffer;
+      // Reserve the memory for 'serialize_stream_' so we don't need to scrounge up
+      // another buffer during spilling.
+      RETURN_IF_ERROR(serialize_stream_->PrepareForWrite(&got_buffer));
+      DCHECK(got_buffer) << "Accounted in min reservation"
+                         << buffer_pool_client()->DebugString();
+      DCHECK(serialize_stream_->has_write_iterator());
+    }
+  }
+  RETURN_IF_ERROR(CreateHashPartitions(0));
+  return Status::OK();
+}
+
+Status GroupingAggregator::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) {
+  if (!partition_eos_) {
+    RETURN_IF_ERROR(GetRowsFromPartition(state, row_batch));
+  }
+  *eos = partition_eos_;
+  return Status::OK();
+}
+
+Status GroupingAggregator::GetRowsFromPartition(
+    RuntimeState* state, RowBatch* row_batch) {
+  DCHECK(!row_batch->AtCapacity());
+  if (output_iterator_.AtEnd()) {
+    // Done with this partition, move onto the next one.
+    if (output_partition_ != nullptr) {
+      output_partition_->Close(false);
+      output_partition_ = nullptr;
+    }
+    if (aggregated_partitions_.empty() && spilled_partitions_.empty()) {
+      // No more partitions, all done.
+      partition_eos_ = true;
+      return Status::OK();
+    }
+    // Process next partition.
+    RETURN_IF_ERROR(NextPartition());
+    DCHECK(output_partition_ != nullptr);
+  }
+
+  SCOPED_TIMER(get_results_timer_);
+
+  // The output row batch may reference memory allocated by Serialize() or Finalize(),
+  // allocating that memory directly from the row batch's pool means we can safely return
+  // the batch.
+  vector<ScopedResultsPool> allocate_from_batch_pool = ScopedResultsPool::Create(
+      output_partition_->agg_fn_evals, row_batch->tuple_data_pool());
+  int count = 0;
+  const int N = BitUtil::RoundUpToPowerOfTwo(state->batch_size());
+  // Keeping returning rows from the current partition.
+  while (!output_iterator_.AtEnd() && !row_batch->AtCapacity()) {
+    // This loop can go on for a long time if the conjuncts are very selective. Do query
+    // maintenance every N iterations.
+    if ((count++ & (N - 1)) == 0) {
+      RETURN_IF_CANCELLED(state);
+      RETURN_IF_ERROR(QueryMaintenance(state));
+    }
+
+    int row_idx = row_batch->AddRow();
+    TupleRow* row = row_batch->GetRow(row_idx);
+    Tuple* intermediate_tuple = output_iterator_.GetTuple();
+    Tuple* output_tuple = GetOutputTuple(output_partition_->agg_fn_evals,
+        intermediate_tuple, row_batch->tuple_data_pool());
+    output_iterator_.Next();
+    row->SetTuple(0, output_tuple);
+    DCHECK_EQ(conjunct_evals_.size(), conjuncts_.size());
+    if (ExecNode::EvalConjuncts(conjunct_evals_.data(), conjuncts_.size(), row)) {
+      row_batch->CommitLastRow();
+      ++num_rows_returned_;
+      if (ReachedLimit()) break;
+    }
+  }
+
+  COUNTER_SET(rows_returned_counter_, num_rows_returned_);
+  partition_eos_ = ReachedLimit();
+  if (output_iterator_.AtEnd()) row_batch->MarkNeedsDeepCopy();
+
+  return Status::OK();
+}
+
+bool GroupingAggregator::ShouldExpandPreaggHashTables() const {
+  int64_t ht_mem = 0;
+  int64_t ht_rows = 0;
+  for (int i = 0; i < PARTITION_FANOUT; ++i) {
+    HashTable* ht = hash_partitions_[i]->hash_tbl.get();
+    ht_mem += ht->CurrentMemSize();
+    ht_rows += ht->size();
+  }
+
+  // Need some rows in tables to have valid statistics.
+  if (ht_rows == 0) return true;
+
+  // Find the appropriate reduction factor in our table for the current hash table sizes.
+  int cache_level = 0;
+  while (cache_level + 1 < STREAMING_HT_MIN_REDUCTION_SIZE
+      && ht_mem >= STREAMING_HT_MIN_REDUCTION[cache_level + 1].min_ht_mem) {
+    ++cache_level;
+  }
+
+  // Compare the number of rows in the hash table with the number of input rows that
+  // were aggregated into it. Exclude passed through rows from this calculation since
+  // they were not in hash tables.
+  const int64_t aggregated_input_rows = num_input_rows_ - num_rows_returned_;
+  const int64_t expected_input_rows = estimated_input_cardinality_ - num_rows_returned_;
+  double current_reduction = static_cast<double>(aggregated_input_rows) / ht_rows;
+
+  // TODO: workaround for IMPALA-2490: subplan node rows_returned counter may be
+  // inaccurate, which could lead to a divide by zero below.
+  if (aggregated_input_rows <= 0) return true;
+
+  // Extrapolate the current reduction factor (r) using the formula
+  // R = 1 + (N / n) * (r - 1), where R is the reduction factor over the full input data
+  // set, N is the number of input rows, excluding passed-through rows, and n is the
+  // number of rows inserted or merged into the hash tables. This is a very rough
+  // approximation but is good enough to be useful.
+  // TODO: consider collecting more statistics to better estimate reduction.
+  double estimated_reduction = aggregated_input_rows >= expected_input_rows ?
+      current_reduction :
+      1 + (expected_input_rows / aggregated_input_rows) * (current_reduction - 1);
+  double min_reduction =
+      STREAMING_HT_MIN_REDUCTION[cache_level].streaming_ht_min_reduction;
+
+  COUNTER_SET(preagg_estimated_reduction_, estimated_reduction);
+  COUNTER_SET(preagg_streaming_ht_min_reduction_, min_reduction);
+  return estimated_reduction > min_reduction;
+}
+
+void GroupingAggregator::CleanupHashTbl(
+    const vector<AggFnEvaluator*>& agg_fn_evals, HashTable::Iterator it) {
+  if (!needs_finalize_ && !needs_serialize_) return;
+
+  // Iterate through the remaining rows in the hash table and call Serialize/Finalize on
+  // them in order to free any memory allocated by UDAs.
+  if (needs_finalize_) {
+    // Finalize() requires a dst tuple but we don't actually need the result,
+    // so allocate a single dummy tuple to avoid accumulating memory.
+    Tuple* dummy_dst = nullptr;
+    dummy_dst = Tuple::Create(output_tuple_desc_->byte_size(), tuple_pool_.get());
+    while (!it.AtEnd()) {
+      Tuple* tuple = it.GetTuple();
+      AggFnEvaluator::Finalize(agg_fn_evals, tuple, dummy_dst);
+      it.Next();
+      // Free any expr result allocations to prevent them accumulating excessively.
+      expr_results_pool_->Clear();
+    }
+  } else {
+    while (!it.AtEnd()) {
+      Tuple* tuple = it.GetTuple();
+      AggFnEvaluator::Serialize(agg_fn_evals, tuple);
+      it.Next();
+      // Free any expr result allocations to prevent them accumulating excessively.
+      expr_results_pool_->Clear();
+    }
+  }
+}
+
+Status GroupingAggregator::Reset(RuntimeState* state) {
+  DCHECK(!is_streaming_preagg_) << "Cannot reset preaggregation";
+  partition_eos_ = false;
+  // Reset the HT and the partitions for this grouping agg.
+  ht_ctx_->set_level(0);
+  ClosePartitions();
+  return Status::OK();
+}
+
+void GroupingAggregator::Close(RuntimeState* state) {
+  // Iterate through the remaining rows in the hash table and call Serialize/Finalize on
+  // them in order to free any memory allocated by UDAs
+  if (output_partition_ != nullptr) {
+    CleanupHashTbl(output_partition_->agg_fn_evals, output_iterator_);
+    output_partition_->Close(false);
+  }
+
+  ClosePartitions();
+
+  if (tuple_pool_.get() != nullptr) tuple_pool_->FreeAll();
+  if (ht_ctx_.get() != nullptr) ht_ctx_->Close(state);
+  ht_ctx_.reset();
+  if (serialize_stream_.get() != nullptr) {
+    serialize_stream_->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+  }
+  ScalarExpr::Close(grouping_exprs_);
+  ScalarExpr::Close(build_exprs_);
+
+  reservation_manager_.Close(state);
+  // Must be called after tuple_pool_ is freed, so that mem_tracker_ can be closed.
+  Aggregator::Close(state);
+}
+
+Status GroupingAggregator::AddBatch(RuntimeState* state, RowBatch* batch) {
+  SCOPED_TIMER(build_timer_);
+  num_input_rows_ += batch->num_rows();
+
+  TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
+  if (add_batch_impl_fn_ != nullptr) {
+    RETURN_IF_ERROR(add_batch_impl_fn_(this, batch, prefetch_mode, ht_ctx_.get()));
+  } else {
+    RETURN_IF_ERROR(AddBatchImpl<false>(batch, prefetch_mode, ht_ctx_.get()));
+  }
+
+  return Status::OK();
+}
+
+Status GroupingAggregator::AddBatchStreaming(
+    RuntimeState* state, RowBatch* out_batch, RowBatch* child_batch) {
+  SCOPED_TIMER(streaming_timer_);
+  num_input_rows_ += child_batch->num_rows();
+
+  int remaining_capacity[PARTITION_FANOUT];
+  bool ht_needs_expansion = false;
+  for (int i = 0; i < PARTITION_FANOUT; ++i) {
+    HashTable* hash_tbl = GetHashTable(i);
+    remaining_capacity[i] = hash_tbl->NumInsertsBeforeResize();
+    ht_needs_expansion |= remaining_capacity[i] < child_batch->num_rows();
+  }
+
+  // Stop expanding hash tables if we're not reducing the input sufficiently. As our
+  // hash tables expand out of each level of cache hierarchy, every hash table lookup
+  // will take longer. We also may not be able to expand hash tables because of memory
+  // pressure. In this case HashTable::CheckAndResize() will fail. In either case we
+  // should always use the remaining space in the hash table to avoid wasting memory.
+  if (ht_needs_expansion && ShouldExpandPreaggHashTables()) {
+    for (int i = 0; i < PARTITION_FANOUT; ++i) {
+      HashTable* ht = GetHashTable(i);
+      if (remaining_capacity[i] < child_batch->num_rows()) {
+        SCOPED_TIMER(ht_resize_timer_);
+        bool resized;
+        RETURN_IF_ERROR(
+            ht->CheckAndResize(child_batch->num_rows(), ht_ctx_.get(), &resized));
+        if (resized) {
+          remaining_capacity[i] = ht->NumInsertsBeforeResize();
+        }
+      }
+    }
+  }
+
+  TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
+  if (add_batch_streaming_impl_fn_ != nullptr) {
+    RETURN_IF_ERROR(add_batch_streaming_impl_fn_(this, needs_serialize_, prefetch_mode,
+        child_batch, out_batch, ht_ctx_.get(), remaining_capacity));
+  } else {
+    RETURN_IF_ERROR(AddBatchStreamingImpl(needs_serialize_, prefetch_mode, child_batch,
+        out_batch, ht_ctx_.get(), remaining_capacity));
+  }
+
+  num_rows_returned_ += out_batch->num_rows();
+  COUNTER_SET(num_passthrough_rows_, num_rows_returned_);
+  return Status::OK();
+}
+
+Status GroupingAggregator::InputDone() {
+  return MoveHashPartitions(num_input_rows_);
+}
+
+Tuple* GroupingAggregator::ConstructIntermediateTuple(
+    const vector<AggFnEvaluator*>& agg_fn_evals, MemPool* pool, Status* status) noexcept {
+  const int fixed_size = intermediate_tuple_desc_->byte_size();
+  const int varlen_size = GroupingExprsVarlenSize();
+  const int tuple_data_size = fixed_size + varlen_size;
+  uint8_t* tuple_data = pool->TryAllocate(tuple_data_size);
+  if (UNLIKELY(tuple_data == nullptr)) {
+    string details = Substitute("Cannot perform aggregation at aggregator with id $0. "
+                                "Failed to allocate $1 bytes for intermediate tuple.",
+        id_, tuple_data_size);
+    *status = pool->mem_tracker()->MemLimitExceeded(state_, details, tuple_data_size);
+    return nullptr;
+  }
+  memset(tuple_data, 0, fixed_size);
+  Tuple* intermediate_tuple = reinterpret_cast<Tuple*>(tuple_data);
+  uint8_t* varlen_data = tuple_data + fixed_size;
+  CopyGroupingValues(intermediate_tuple, varlen_data, varlen_size);
+  InitAggSlots(agg_fn_evals, intermediate_tuple);
+  return intermediate_tuple;
+}
+
+Tuple* GroupingAggregator::ConstructIntermediateTuple(
+    const vector<AggFnEvaluator*>& agg_fn_evals, BufferedTupleStream* stream,
+    Status* status) noexcept {
+  DCHECK(stream != nullptr && status != nullptr);
+  // Allocate space for the entire tuple in the stream.
+  const int fixed_size = intermediate_tuple_desc_->byte_size();
+  const int varlen_size = GroupingExprsVarlenSize();
+  const int tuple_size = fixed_size + varlen_size;
+  uint8_t* tuple_data = stream->AddRowCustomBegin(tuple_size, status);
+  if (UNLIKELY(tuple_data == nullptr)) {
+    // If we failed to allocate and did not hit an error (indicated by a non-ok status),
+    // the caller of this function can try to free some space, e.g. through spilling, and
+    // re-attempt to allocate space for this row.
+    return nullptr;
+  }
+  Tuple* tuple = reinterpret_cast<Tuple*>(tuple_data);
+  tuple->Init(fixed_size);
+  uint8_t* varlen_buffer = tuple_data + fixed_size;
+  CopyGroupingValues(tuple, varlen_buffer, varlen_size);
+  InitAggSlots(agg_fn_evals, tuple);
+  stream->AddRowCustomEnd(tuple_size);
+  return tuple;
+}
+
+int GroupingAggregator::GroupingExprsVarlenSize() {
+  int varlen_size = 0;
+  // TODO: The hash table could compute this as it hashes.
+  for (int expr_idx : string_grouping_exprs_) {
+    StringValue* sv = reinterpret_cast<StringValue*>(ht_ctx_->ExprValue(expr_idx));
+    // Avoid branching by multiplying length by null bit.
+    varlen_size += sv->len * !ht_ctx_->ExprValueNull(expr_idx);
+  }
+  return varlen_size;
+}
+
+// TODO: codegen this function.
+void GroupingAggregator::CopyGroupingValues(
+    Tuple* intermediate_tuple, uint8_t* buffer, int varlen_size) {
+  // Copy over all grouping slots (the variable length data is copied below).
+  for (int i = 0; i < grouping_exprs_.size(); ++i) {
+    SlotDescriptor* slot_desc = intermediate_tuple_desc_->slots()[i];
+    if (ht_ctx_->ExprValueNull(i)) {
+      intermediate_tuple->SetNull(slot_desc->null_indicator_offset());
+    } else {
+      void* src = ht_ctx_->ExprValue(i);
+      void* dst = intermediate_tuple->GetSlot(slot_desc->tuple_offset());
+      memcpy(dst, src, slot_desc->slot_size());
+    }
+  }
+
+  for (int expr_idx : string_grouping_exprs_) {
+    if (ht_ctx_->ExprValueNull(expr_idx)) continue;
+
+    SlotDescriptor* slot_desc = intermediate_tuple_desc_->slots()[expr_idx];
+    // ptr and len were already copied to the fixed-len part of string value
+    StringValue* sv = reinterpret_cast<StringValue*>(
+        intermediate_tuple->GetSlot(slot_desc->tuple_offset()));
+    memcpy(buffer, sv->ptr, sv->len);
+    sv->ptr = reinterpret_cast<char*>(buffer);
+    buffer += sv->len;
+  }
+}
+
+template <bool AGGREGATED_ROWS>
+Status GroupingAggregator::AppendSpilledRow(
+    Partition* __restrict__ partition, TupleRow* __restrict__ row) {
+  DCHECK(!is_streaming_preagg_);
+  DCHECK(partition->is_spilled());
+  BufferedTupleStream* stream = AGGREGATED_ROWS ?
+      partition->aggregated_row_stream.get() :
+      partition->unaggregated_row_stream.get();
+  DCHECK(!stream->is_pinned());
+  Status status;
+  if (LIKELY(stream->AddRow(row, &status))) return Status::OK();
+  RETURN_IF_ERROR(status);
+
+  // Keep trying to free memory by spilling until we succeed or hit an error.
+  // Running out of partitions to spill is treated as an error by SpillPartition().
+  while (true) {
+    RETURN_IF_ERROR(SpillPartition(AGGREGATED_ROWS));
+    if (stream->AddRow(row, &status)) return Status::OK();
+    RETURN_IF_ERROR(status);
+  }
+}
+
+void GroupingAggregator::SetDebugOptions(const TDebugOptions& debug_options) {
+  debug_options_ = debug_options;
+}
+
+string GroupingAggregator::DebugString(int indentation_level) const {
+  stringstream ss;
+  DebugString(indentation_level, &ss);
+  return ss.str();
+}
+
+void GroupingAggregator::DebugString(int indentation_level, stringstream* out) const {
+  *out << string(indentation_level * 2, ' ');
+  *out << "GroupingAggregator("
+       << "intermediate_tuple_id=" << intermediate_tuple_id_
+       << " output_tuple_id=" << output_tuple_id_ << " needs_finalize=" << needs_finalize_
+       << " grouping_exprs=" << ScalarExpr::DebugString(grouping_exprs_)
+       << " agg_exprs=" << AggFn::DebugString(agg_fns_);
+  *out << ")";
+}
+
+Status GroupingAggregator::CreateHashPartitions(int level, int single_partition_idx) {
+  if (is_streaming_preagg_) DCHECK_EQ(level, 0);
+  if (UNLIKELY(level >= MAX_PARTITION_DEPTH)) {
+    return Status(
+        TErrorCode::PARTITIONED_AGG_MAX_PARTITION_DEPTH, id_, MAX_PARTITION_DEPTH);
+  }
+  ht_ctx_->set_level(level);
+
+  DCHECK(hash_partitions_.empty());
+  int num_partitions_created = 0;
+  for (int i = 0; i < PARTITION_FANOUT; ++i) {
+    hash_tbls_[i] = nullptr;
+    if (single_partition_idx == -1 || i == single_partition_idx) {
+      Partition* new_partition = partition_pool_->Add(new Partition(this, level, i));
+      ++num_partitions_created;
+      hash_partitions_.push_back(new_partition);
+      RETURN_IF_ERROR(new_partition->InitStreams());
+    } else {
+      hash_partitions_.push_back(nullptr);
+    }
+  }
+  // Now that all the streams are reserved (meaning we have enough memory to execute
+  // the algorithm), allocate the hash tables. These can fail and we can still continue.
+  for (int i = 0; i < PARTITION_FANOUT; ++i) {
+    Partition* partition = hash_partitions_[i];
+    if (partition == nullptr) continue;
+    if (partition->aggregated_row_stream == nullptr) {
+      // Failed to create the aggregated row stream - cannot create a hash table.
+      // Just continue with a NULL hash table so rows will be passed through.
+      DCHECK(is_streaming_preagg_);
+    } else {
+      bool got_memory;
+      RETURN_IF_ERROR(partition->InitHashTable(&got_memory));
+      // Spill the partition if we cannot create a hash table for a merge aggregation.
+      if (UNLIKELY(!got_memory)) {
+        DCHECK(!is_streaming_preagg_) << "Preagg reserves enough memory for hash tables";
+        // If we're repartitioning, we will be writing aggregated rows first.
+        RETURN_IF_ERROR(partition->Spill(level > 0));
+      }
+    }
+    hash_tbls_[i] = partition->hash_tbl.get();
+  }
+  // In this case we did not have to repartition, so ensure that while building the hash
+  // table all rows will be inserted into the partition at 'single_partition_idx' in case
+  // a non deterministic grouping expression causes a row to hash to a different
+  // partition index.
+  if (single_partition_idx != -1) {
+    Partition* partition = hash_partitions_[single_partition_idx];
+    for (int i = 0; i < PARTITION_FANOUT; ++i) {
+      hash_partitions_[i] = partition;
+      hash_tbls_[i] = partition->hash_tbl.get();
+    }
+  }
+
+  COUNTER_ADD(partitions_created_, num_partitions_created);
+  if (!is_streaming_preagg_) {
+    COUNTER_SET(max_partition_level_, level);
+  }
+  return Status::OK();
+}
+
+Status GroupingAggregator::CheckAndResizeHashPartitions(
+    bool partitioning_aggregated_rows, int num_rows, const HashTableCtx* ht_ctx) {
+  DCHECK(!is_streaming_preagg_);
+  for (int i = 0; i < PARTITION_FANOUT; ++i) {
+    Partition* partition = hash_partitions_[i];
+    if (partition == nullptr) continue;
+    while (!partition->is_spilled()) {
+      {
+        SCOPED_TIMER(ht_resize_timer_);
+        bool resized;
+        RETURN_IF_ERROR(partition->hash_tbl->CheckAndResize(num_rows, ht_ctx, &resized));
+        if (resized) break;
+      }
+      RETURN_IF_ERROR(SpillPartition(partitioning_aggregated_rows));
+    }
+  }
+  return Status::OK();
+}
+
+Status GroupingAggregator::NextPartition() {
+  DCHECK(output_partition_ == nullptr);
+
+  if (!is_in_subplan_ && spilled_partitions_.empty()) {
+    // All partitions are in memory. Release reservation that was used for previous
+    // partitions that is no longer needed. If we have spilled partitions, we want to
+    // hold onto all reservation in case it is needed to process the spilled partitions.
+    DCHECK(!buffer_pool_client()->has_unpinned_pages());
+    Status status = reservation_manager_.ReleaseUnusedReservation();
+    DCHECK(status.ok()) << "Should not fail - all partitions are in memory so there are "
+                        << "no unpinned pages. " << status.GetDetail();
+  }
+
+  // Keep looping until we get to a partition that fits in memory.
+  Partition* partition = nullptr;
+  while (true) {
+    // First return partitions that are fully aggregated (and in memory).
+    if (!aggregated_partitions_.empty()) {
+      partition = aggregated_partitions_.front();
+      DCHECK(!partition->is_spilled());
+      aggregated_partitions_.pop_front();
+      break;
+    }
+
+    // No aggregated partitions in memory - we should not be using any reservation aside
+    // from 'serialize_stream_'.
+    DCHECK_EQ(serialize_stream_ != nullptr ? serialize_stream_->BytesPinned(false) : 0,
+        buffer_pool_client()->GetUsedReservation())
+        << buffer_pool_client()->DebugString();
+
+    // Try to fit a single spilled partition in memory. We can often do this because
+    // we only need to fit 1/PARTITION_FANOUT of the data in memory.
+    // TODO: in some cases when the partition probably won't fit in memory it could
+    // be better to skip directly to repartitioning.
+    RETURN_IF_ERROR(BuildSpilledPartition(&partition));
+    if (partition != nullptr) break;
+
+    // If we can't fit the partition in memory, repartition it.
+    RETURN_IF_ERROR(RepartitionSpilledPartition());
+  }
+  DCHECK(!partition->is_spilled());
+  DCHECK(partition->hash_tbl.get() != nullptr);
+  DCHECK(partition->aggregated_row_stream->is_pinned());
+
+  output_partition_ = partition;
+  output_iterator_ = output_partition_->hash_tbl->Begin(ht_ctx_.get());
+  COUNTER_ADD(num_hash_buckets_, output_partition_->hash_tbl->num_buckets());
+  return Status::OK();
+}
+
+Status GroupingAggregator::BuildSpilledPartition(Partition** built_partition) {
+  DCHECK(!spilled_partitions_.empty());
+  DCHECK(!is_streaming_preagg_);
+  // Leave the partition in 'spilled_partitions_' to be closed if we hit an error.
+  Partition* src_partition = spilled_partitions_.front();
+  DCHECK(src_partition->is_spilled());
+
+  // Create a new hash partition from the rows of the spilled partition. This is simpler
+  // than trying to finish building a partially-built partition in place. We only
+  // initialise one hash partition that all rows in 'src_partition' will hash to.
+  RETURN_IF_ERROR(CreateHashPartitions(src_partition->level, src_partition->idx));
+  Partition* dst_partition = hash_partitions_[src_partition->idx];
+  DCHECK(dst_partition != nullptr);
+
+  // Rebuild the hash table over spilled aggregate rows then start adding unaggregated
+  // rows to the hash table. It's possible the partition will spill at either stage.
+  // In that case we need to finish processing 'src_partition' so that all rows are
+  // appended to 'dst_partition'.
+  // TODO: if the partition spills again but the aggregation reduces the input
+  // significantly, we could do better here by keeping the incomplete hash table in
+  // memory and only spilling unaggregated rows that didn't fit in the hash table
+  // (somewhat similar to the passthrough pre-aggregation).
+  RETURN_IF_ERROR(ProcessStream<true>(src_partition->aggregated_row_stream.get()));
+  RETURN_IF_ERROR(ProcessStream<false>(src_partition->unaggregated_row_stream.get()));
+  src_partition->Close(false);
+  spilled_partitions_.pop_front();
+  hash_partitions_.clear();
+
+  if (dst_partition->is_spilled()) {
+    PushSpilledPartition(dst_partition);
+    *built_partition = nullptr;
+    // Spilled the partition - we should not be using any reservation except from
+    // 'serialize_stream_'.
+    DCHECK_EQ(serialize_stream_ != nullptr ? serialize_stream_->BytesPinned(false) : 0,
+        buffer_pool_client()->GetUsedReservation())
+        << buffer_pool_client()->DebugString();
+  } else {
+    *built_partition = dst_partition;
+  }
+  return Status::OK();
+}
+
+Status GroupingAggregator::RepartitionSpilledPartition() {
+  DCHECK(!spilled_partitions_.empty());
+  DCHECK(!is_streaming_preagg_);
+  // Leave the partition in 'spilled_partitions_' to be closed if we hit an error.
+  Partition* partition = spilled_partitions_.front();
+  DCHECK(partition->is_spilled());
+
+  // Create the new hash partitions to repartition into. This will allocate a
+  // write buffer for each partition's aggregated row stream.
+  RETURN_IF_ERROR(CreateHashPartitions(partition->level + 1));
+  COUNTER_ADD(num_repartitions_, 1);
+
+  // Rows in this partition could have been spilled into two streams, depending
+  // on if it is an aggregated intermediate, or an unaggregated row. Aggregated
+  // rows are processed first to save a hash table lookup in AddBatchImpl().
+  RETURN_IF_ERROR(ProcessStream<true>(partition->aggregated_row_stream.get()));
+
+  // Prepare write buffers so we can append spilled rows to unaggregated partitions.
+  for (Partition* hash_partition : hash_partitions_) {
+    if (!hash_partition->is_spilled()) continue;
+    // The aggregated rows have been repartitioned. Free up at least a buffer's worth of
+    // reservation and use it to pin the unaggregated write buffer.
+    hash_partition->aggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL);
+    bool got_buffer;
+    RETURN_IF_ERROR(
+        hash_partition->unaggregated_row_stream->PrepareForWrite(&got_buffer));
+    DCHECK(got_buffer) << "Accounted in min reservation"
+                       << buffer_pool_client()->DebugString();
+  }
+  RETURN_IF_ERROR(ProcessStream<false>(partition->unaggregated_row_stream.get()));
+
+  COUNTER_ADD(num_row_repartitioned_, partition->aggregated_row_stream->num_rows());
+  COUNTER_ADD(num_row_repartitioned_, partition->unaggregated_row_stream->num_rows());
+
+  partition->Close(false);
+  spilled_partitions_.pop_front();
+
+  // Done processing this partition. Move the new partitions into
+  // spilled_partitions_/aggregated_partitions_.
+  int64_t num_input_rows = partition->aggregated_row_stream->num_rows()
+      + partition->unaggregated_row_stream->num_rows();
+  RETURN_IF_ERROR(MoveHashPartitions(num_input_rows));
+  return Status::OK();
+}
+
+template <bool AGGREGATED_ROWS>
+Status GroupingAggregator::ProcessStream(BufferedTupleStream* input_stream) {
+  DCHECK(!is_streaming_preagg_);
+  if (input_stream->num_rows() > 0) {
+    while (true) {
+      bool got_buffer = false;
+      RETURN_IF_ERROR(input_stream->PrepareForRead(true, &got_buffer));
+      if (got_buffer) break;
+      // Did not have a buffer to read the input stream. Spill and try again.
+      RETURN_IF_ERROR(SpillPartition(AGGREGATED_ROWS));
+    }
+
+    TPrefetchMode::type prefetch_mode = state_->query_options().prefetch_mode;
+    bool eos = false;
+    const RowDescriptor* desc =
+        AGGREGATED_ROWS ? &intermediate_row_desc_ : &input_row_desc_;
+    RowBatch batch(desc, state_->batch_size(), mem_tracker_.get());
+    do {
+      RETURN_IF_ERROR(input_stream->GetNext(&batch, &eos));
+      RETURN_IF_ERROR(
+          AddBatchImpl<AGGREGATED_ROWS>(&batch, prefetch_mode, ht_ctx_.get()));
+      RETURN_IF_ERROR(QueryMaintenance(state_));
+      batch.Reset();
+    } while (!eos);
+  }
+  input_stream->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
+  return Status::OK();
+}
+
+Status GroupingAggregator::SpillPartition(bool more_aggregate_rows) {
+  int64_t max_freed_mem = 0;
+  int partition_idx = -1;
+
+  // Iterate over the partitions and pick the largest partition that is not spilled.
+  for (int i = 0; i < hash_partitions_.size(); ++i) {
+    if (hash_partitions_[i] == nullptr) continue;
+    if (hash_partitions_[i]->is_closed) continue;
+    if (hash_partitions_[i]->is_spilled()) continue;
+    // Pass 'true' because we need to keep the write block pinned. See Partition::Spill().
+    int64_t mem = hash_partitions_[i]->aggregated_row_stream->BytesPinned(true);
+    mem += hash_partitions_[i]->hash_tbl->ByteSize();
+    mem += hash_partitions_[i]->agg_fn_perm_pool->total_reserved_bytes();
+    DCHECK_GT(mem, 0); // At least the hash table buckets should occupy memory.
+    if (mem > max_freed_mem) {
+      max_freed_mem = mem;
+      partition_idx = i;
+    }
+  }
+  DCHECK_NE(partition_idx, -1) << "Should have been able to spill a partition to "
+                               << "reclaim memory: "
+                               << buffer_pool_client()->DebugString();
+  // Remove references to the destroyed hash table from 'hash_tbls_'.
+  // Additionally, we might be dealing with a rebuilt spilled partition, where all
+  // partitions point to a single in-memory partition. This also ensures that 'hash_tbls_'
+  // remains consistent in that case.
+  for (int i = 0; i < PARTITION_FANOUT; ++i) {
+    if (hash_partitions_[i] == hash_partitions_[partition_idx]) hash_tbls_[i] = nullptr;
+  }
+  return hash_partitions_[partition_idx]->Spill(more_aggregate_rows);
+}
+
+Status GroupingAggregator::MoveHashPartitions(int64_t num_input_rows) {
+  DCHECK(!hash_partitions_.empty());
+  stringstream ss;
+  ss << "PA(node_id=" << id_ << ") partitioned(level=" << hash_partitions_[0]->level
+     << ") " << num_input_rows << " rows into:" << endl;
+  for (int i = 0; i < hash_partitions_.size(); ++i) {
+    Partition* partition = hash_partitions_[i];
+    if (partition == nullptr) continue;
+    // We might be dealing with a rebuilt spilled partition, where all partitions are
+    // pointing to a single in-memory partition, so make sure we only proceed for the
+    // right partition.
+    if (i != partition->idx) continue;
+    int64_t aggregated_rows = 0;
+    if (partition->aggregated_row_stream != nullptr) {
+      aggregated_rows = partition->aggregated_row_stream->num_rows();
+    }
+    int64_t unaggregated_rows = 0;
+    if (partition->unaggregated_row_stream != nullptr) {
+      unaggregated_rows = partition->unaggregated_row_stream->num_rows();
+    }
+    double total_rows = aggregated_rows + unaggregated_rows;
+    double percent = total_rows * 100 / num_input_rows;
+    ss << "  " << i << " " << (partition->is_spilled() ? "spilled" : "not spilled")
+       << " (fraction=" << fixed << setprecision(2) << percent << "%)" << endl
+       << "    #aggregated rows:" << aggregated_rows << endl
+       << "    #unaggregated rows: " << unaggregated_rows << endl;
+
+    // TODO: update counters to support doubles.
+    COUNTER_SET(largest_partition_percent_, static_cast<int64_t>(percent));
+
+    if (total_rows == 0) {
+      partition->Close(false);
+    } else if (partition->is_spilled()) {
+      PushSpilledPartition(partition);
+    } else {
+      aggregated_partitions_.push_back(partition);
+    }
+  }
+  VLOG(2) << ss.str();
+  hash_partitions_.clear();
+  return Status::OK();
+}
+
+void GroupingAggregator::PushSpilledPartition(Partition* partition) {
+  DCHECK(partition->is_spilled());
+  DCHECK(partition->hash_tbl == nullptr);
+  // Ensure all pages in the spilled partition's streams are unpinned by invalidating
+  // the streams' read and write iterators. We may need all the memory to process the
+  // next spilled partitions.
+  partition->aggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL);
+  partition->unaggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL);
+  spilled_partitions_.push_front(partition);
+}
+
+void GroupingAggregator::ClosePartitions() {
+  for (Partition* partition : hash_partitions_) {
+    if (partition != nullptr) partition->Close(true);
+  }
+  hash_partitions_.clear();
+  for (Partition* partition : aggregated_partitions_) partition->Close(true);
+  aggregated_partitions_.clear();
+  for (Partition* partition : spilled_partitions_) partition->Close(true);
+  spilled_partitions_.clear();
+  memset(hash_tbls_, 0, sizeof(hash_tbls_));
+  partition_pool_->Clear();
+}
+
+int64_t GroupingAggregator::MinReservation() const {
+  // Must be kept in sync with AggregationNode.computeNodeResourceProfile() in fe.
+  if (is_streaming_preagg_) {
+    // Reserve at least one buffer and a 64kb hash table per partition.
+    return (resource_profile_.spillable_buffer_size + 64 * 1024) * PARTITION_FANOUT;
+  }
+  int num_buffers = PARTITION_FANOUT + 1 + (needs_serialize_ ? 1 : 0);
+  // Two of the buffers must fit the maximum row.
+  return resource_profile_.spillable_buffer_size * (num_buffers - 2)
+      + resource_profile_.max_row_buffer_size * 2;
+}
+
+Status GroupingAggregator::QueryMaintenance(RuntimeState* state) {
+  expr_results_pool_->Clear();
+  return state->CheckQueryState();
+}
+
+BufferPool::ClientHandle* GroupingAggregator::buffer_pool_client() {
+  return reservation_manager_.buffer_pool_client();
+}
+
+Status GroupingAggregator::CodegenAddBatchImpl(
+    LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) {
+  llvm::Function* update_tuple_fn;
+  RETURN_IF_ERROR(CodegenUpdateTuple(codegen, &update_tuple_fn));
+
+  // Get the cross compiled update row batch function
+  IRFunction::Type ir_fn = IRFunction::GROUPING_AGG_ADD_BATCH_IMPL;
+  llvm::Function* add_batch_impl_fn = codegen->GetFunction(ir_fn, true);
+  DCHECK(add_batch_impl_fn != nullptr);
+
+  int replaced;
+  // Codegen for grouping using hash table
+
+  // Replace prefetch_mode with constant so branches can be optimised out.
+  llvm::Value* prefetch_mode_arg = codegen->GetArgument(add_batch_impl_fn, 3);
+  prefetch_mode_arg->replaceAllUsesWith(codegen->GetI32Constant(prefetch_mode));
+
+  // The codegen'd AddBatchImpl function is only used in Open() with level_ = 0,
+  // so don't use murmur hash
+  llvm::Function* hash_fn;
+  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, /* use murmur */ false, &hash_fn));
+
+  // Codegen HashTable::Equals<true>
+  llvm::Function* build_equals_fn;
+  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, true, &build_equals_fn));
+
+  // Codegen for evaluating input rows
+  llvm::Function* eval_grouping_expr_fn;
+  RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(codegen, false, &eval_grouping_expr_fn));
+
+  // Replace call sites
+  replaced =
+      codegen->ReplaceCallSites(add_batch_impl_fn, eval_grouping_expr_fn, "EvalProbeRow");
+  DCHECK_EQ(replaced, 1);
+
+  replaced = codegen->ReplaceCallSites(add_batch_impl_fn, hash_fn, "HashRow");
+  DCHECK_EQ(replaced, 1);
+
+  replaced = codegen->ReplaceCallSites(add_batch_impl_fn, build_equals_fn, "Equals");
+  DCHECK_EQ(replaced, 1);
+
+  HashTableCtx::HashTableReplacedConstants replaced_constants;
+  const bool stores_duplicates = false;
+  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(
+      codegen, stores_duplicates, 1, add_batch_impl_fn, &replaced_constants));
+  DCHECK_GE(replaced_constants.stores_nulls, 1);
+  DCHECK_GE(replaced_constants.finds_some_nulls, 1);
+  DCHECK_GE(replaced_constants.stores_duplicates, 1);
+  DCHECK_GE(replaced_constants.stores_tuples, 1);
+  DCHECK_GE(replaced_constants.quadratic_probing, 1);
+
+  replaced = codegen->ReplaceCallSites(add_batch_impl_fn, update_tuple_fn, "UpdateTuple");
+  DCHECK_GE(replaced, 1);
+  add_batch_impl_fn = codegen->FinalizeFunction(add_batch_impl_fn);
+  if (add_batch_impl_fn == nullptr) {
+    return Status("GroupingAggregator::CodegenAddBatchImpl(): codegen'd "
+                  "AddBatchImpl() function failed verification, see log");
+  }
+
+  void** codegened_fn_ptr = reinterpret_cast<void**>(&add_batch_impl_fn_);
+  codegen->AddFunctionToJit(add_batch_impl_fn, codegened_fn_ptr);
+  return Status::OK();
+}
+
+Status GroupingAggregator::CodegenAddBatchStreamingImpl(
+    LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) {
+  DCHECK(is_streaming_preagg_);
+
+  IRFunction::Type ir_fn = IRFunction::GROUPING_AGG_ADD_BATCH_STREAMING_IMPL;
+  llvm::Function* add_batch_streaming_impl_fn = codegen->GetFunction(ir_fn, true);
+  DCHECK(add_batch_streaming_impl_fn != nullptr);
+
+  // Make needs_serialize arg constant so dead code can be optimised out.
+  llvm::Value* needs_serialize_arg = codegen->GetArgument(add_batch_streaming_impl_fn, 2);
+  needs_serialize_arg->replaceAllUsesWith(codegen->GetBoolConstant(needs_serialize_));
+
+  // Replace prefetch_mode with constant so branches can be optimised out.
+  llvm::Value* prefetch_mode_arg = codegen->GetArgument(add_batch_streaming_impl_fn, 3);
+  prefetch_mode_arg->replaceAllUsesWith(codegen->GetI32Constant(prefetch_mode));
+
+  llvm::Function* update_tuple_fn;
+  RETURN_IF_ERROR(CodegenUpdateTuple(codegen, &update_tuple_fn));
+
+  // We only use the top-level hash function for streaming aggregations.
+  llvm::Function* hash_fn;
+  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, false, &hash_fn));
+
+  // Codegen HashTable::Equals
+  llvm::Function* equals_fn;
+  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, true, &equals_fn));
+
+  // Codegen for evaluating input rows
+  llvm::Function* eval_grouping_expr_fn;
+  RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(codegen, false, &eval_grouping_expr_fn));
+
+  // Replace call sites
+  int replaced = codegen->ReplaceCallSites(
+      add_batch_streaming_impl_fn, update_tuple_fn, "UpdateTuple");
+  DCHECK_EQ(replaced, 2);
+
+  replaced = codegen->ReplaceCallSites(
+      add_batch_streaming_impl_fn, eval_grouping_expr_fn, "EvalProbeRow");
+  DCHECK_EQ(replaced, 1);
+
+  replaced = codegen->ReplaceCallSites(add_batch_streaming_impl_fn, hash_fn, "HashRow");
+  DCHECK_EQ(replaced, 1);
+
+  replaced = codegen->ReplaceCallSites(add_batch_streaming_impl_fn, equals_fn, "Equals");
+  DCHECK_EQ(replaced, 1);
+
+  HashTableCtx::HashTableReplacedConstants replaced_constants;
+  const bool stores_duplicates = false;
+  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(
+      codegen, stores_duplicates, 1, add_batch_streaming_impl_fn, &replaced_constants));
+  DCHECK_GE(replaced_constants.stores_nulls, 1);
+  DCHECK_GE(replaced_constants.finds_some_nulls, 1);
+  DCHECK_GE(replaced_constants.stores_duplicates, 1);
+  DCHECK_GE(replaced_constants.stores_tuples, 1);
+  DCHECK_GE(replaced_constants.quadratic_probing, 1);
+
+  DCHECK(add_batch_streaming_impl_fn != nullptr);
+  add_batch_streaming_impl_fn = codegen->FinalizeFunction(add_batch_streaming_impl_fn);
+  if (add_batch_streaming_impl_fn == nullptr) {
+    return Status("GroupingAggregator::CodegenAddBatchStreamingImpl(): codegen'd "
+                  "AddBatchStreamingImpl() function failed verification, see log");
+  }
+
+  codegen->AddFunctionToJit(add_batch_streaming_impl_fn,
+      reinterpret_cast<void**>(&add_batch_streaming_impl_fn_));
+  return Status::OK();
+}
+
+// Instantiate required templates.
+template Status GroupingAggregator::AppendSpilledRow<false>(Partition*, TupleRow*);
+template Status GroupingAggregator::AppendSpilledRow<true>(Partition*, TupleRow*);
+} // namespace impala
diff --git a/be/src/exec/partitioned-aggregation-node.h b/be/src/exec/grouping-aggregator.h
similarity index 71%
rename from be/src/exec/partitioned-aggregation-node.h
rename to be/src/exec/grouping-aggregator.h
index c522c1b..0d1b893 100644
--- a/be/src/exec/partitioned-aggregation-node.h
+++ b/be/src/exec/grouping-aggregator.h
@@ -15,44 +15,31 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
-#ifndef IMPALA_EXEC_PARTITIONED_AGGREGATION_NODE_H
-#define IMPALA_EXEC_PARTITIONED_AGGREGATION_NODE_H
+#ifndef IMPALA_EXEC_GROUPING_AGGREGATOR_H
+#define IMPALA_EXEC_GROUPING_AGGREGATOR_H
 
 #include <deque>
+#include <memory>
+#include <vector>
 
-#include <boost/scoped_ptr.hpp>
-
-#include "exec/exec-node.h"
+#include "exec/aggregator.h"
 #include "exec/hash-table.h"
 #include "runtime/buffered-tuple-stream.h"
 #include "runtime/bufferpool/suballocator.h"
-#include "runtime/descriptors.h" // for TupleId
+#include "runtime/descriptors.h"
 #include "runtime/mem-pool.h"
-#include "runtime/string-value.h"
-
-namespace llvm {
-class BasicBlock;
-class Function;
-class Value;
-}
+#include "runtime/reservation-manager.h"
 
 namespace impala {
 
-class AggFn;
 class AggFnEvaluator;
-class CodegenAnyVal;
 class LlvmCodeGen;
-class LlvmBuilder;
 class RowBatch;
 class RuntimeState;
-struct StringValue;
 class Tuple;
-class TupleDescriptor;
-class SlotDescriptor;
 
-/// Node for doing partitioned hash aggregation.
-/// This node consumes the input (which can be from the child(0) or a spilled partition).
+/// Aggregator for doing grouping aggregations. Input is passed to the aggregator through
+/// AddBatch(), or AddBatchStreaming() if this is a pre-agg. Then:
 ///  1. Each row is hashed and we pick a dst partition (hash_partitions_).
 ///  2. If the dst partition is not spilled, we probe into the partitions hash table
 ///  to aggregate/insert the row.
@@ -90,7 +77,7 @@ class SlotDescriptor;
 ///
 /// Two-phase aggregation: we support two-phase distributed aggregations, where
 /// pre-aggregrations attempt to reduce the size of data before shuffling data across the
-/// network to be merged by the merge aggregation node. This exec node supports a
+/// network to be merged by the merge aggregation node. This aggregator supports a
 /// streaming mode for pre-aggregations where it maintains a hash table of aggregated
 /// rows, but can pass through unaggregated rows (after transforming them into the
 /// same tuple format as aggregated rows) when a heuristic determines that it is better
@@ -101,10 +88,6 @@ class SlotDescriptor;
 /// TODO: make this less of a heuristic by factoring in the cost of the exchange vs the
 /// cost of the pre-aggregation.
 ///
-/// If there are no grouping expressions, there is only a single output row for both
-/// preaggregations and merge aggregations. This case is handled separately to avoid
-/// building hash tables. There is also no need to do streaming preaggregations.
-///
 /// Handling memory pressure: the node uses two different strategies for responding to
 /// memory pressure, depending on whether it is a streaming pre-aggregation or not. If
 /// the node is a streaming preaggregation, it stops growing its hash table further by
@@ -128,24 +111,34 @@ class SlotDescriptor;
 /// There are so many contexts in use that a plain "ctx" variable should never be used.
 /// Likewise, it's easy to mixup the agg fn ctxs, there should be a way to simplify this.
 /// TODO: support an Init() method with an initial value in the UDAF interface.
-class PartitionedAggregationNode : public ExecNode {
+class GroupingAggregator : public Aggregator {
  public:
-  PartitionedAggregationNode(ObjectPool* pool,
-      const TPlanNode& tnode, const DescriptorTbl& descs);
+  GroupingAggregator(ExecNode* exec_node, ObjectPool* pool, const TPlanNode& tnode,
+      const DescriptorTbl& descs);
 
-  virtual Status Init(const TPlanNode& tnode, RuntimeState* state);
-  virtual Status Prepare(RuntimeState* state);
-  virtual void Codegen(RuntimeState* state);
-  virtual Status Open(RuntimeState* state);
-  virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos);
-  virtual Status Reset(RuntimeState* state);
-  virtual void Close(RuntimeState* state);
+  virtual Status Init(const TPlanNode& tnode, RuntimeState* state) override;
+  virtual Status Prepare(RuntimeState* state) override;
+  virtual void Codegen(RuntimeState* state) override;
+  virtual Status Open(RuntimeState* state) override;
+  virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
+  virtual Status Reset(RuntimeState* state) override;
+  virtual void Close(RuntimeState* state) override;
 
-  static const char* LLVM_CLASS_NAME;
+  virtual Status AddBatch(RuntimeState* state, RowBatch* batch) override;
+  /// Used to insert input rows if this is a streaming pre-agg. Tries to aggregate all of
+  /// the rows of 'child_batch', but if there isn't enough memory available rows will be
+  /// streamed through and returned in 'out_batch'. AddBatch() and AddBatchStreaming()
+  /// should not be called on the same GroupingAggregator.
+  Status AddBatchStreaming(
+      RuntimeState* state, RowBatch* out_batch, RowBatch* child_batch);
+  virtual Status InputDone() override WARN_UNUSED_RESULT;
 
- protected:
-  virtual std::string DebugString(int indentation_level) const;
-  virtual void DebugString(int indentation_level, std::stringstream* out) const;
+  virtual int num_grouping_exprs() override { return grouping_exprs_.size(); }
+
+  virtual void SetDebugOptions(const TDebugOptions& debug_options) override;
+
+  virtual std::string DebugString(int indentation_level = 0) const override;
+  virtual void DebugString(int indentation_level, std::stringstream* out) const override;
 
  private:
   struct Partition;
@@ -177,11 +170,7 @@ class PartitionedAggregationNode : public ExecNode {
   /// doesn't know how to deal with the LLVM IR 'invoke' instruction. Workaround that by
   /// placing the Status here so exceptions won't need to destruct it.
   /// TODO: fix IMPALA-1948 and remove this.
-  Status process_batch_status_;
-
-  /// Tuple into which Update()/Merge()/Serialize() results are stored.
-  TupleId intermediate_tuple_id_;
-  TupleDescriptor* intermediate_tuple_desc_;
+  Status add_batch_status_;
 
   /// Row with the intermediate tuple as its only tuple.
   /// Construct a new row desc for preparing the build exprs because neither the child's
@@ -190,17 +179,6 @@ class PartitionedAggregationNode : public ExecNode {
   /// Lives in the query state's obj_pool.
   RowDescriptor intermediate_row_desc_;
 
-  /// Tuple into which Finalize() results are stored. Possibly the same as
-  /// the intermediate tuple.
-  TupleId output_tuple_id_;
-  TupleDescriptor* output_tuple_desc_;
-
-  /// Certain aggregates require a finalize step, which is the final step of the
-  /// aggregate after consuming all input rows. The finalize step converts the aggregate
-  /// value into its final form. This is true if this node contains aggregate that
-  /// requires a finalize step.
-  const bool needs_finalize_;
-
   /// True if this is first phase of a two-phase distributed aggregation for which we
   /// are doing a streaming preaggregation.
   const bool is_streaming_preagg_;
@@ -208,18 +186,6 @@ class PartitionedAggregationNode : public ExecNode {
   /// True if any of the evaluators require the serialize step.
   bool needs_serialize_;
 
-  /// The list of all aggregate operations for this exec node.
-  std::vector<AggFn*> agg_fns_;
-
-  /// Evaluators for each aggregate function. If this is a grouping aggregation, these
-  /// evaluators are only used to create cloned per-partition evaluators. The cloned
-  /// evaluators are then used to evaluate the functions. If this is a non-grouping
-  /// aggregation these evaluators are used directly to evaluate the functions.
-  ///
-  /// Permanent and result allocations for these allocators are allocated from
-  /// 'expr_perm_pool_' and 'expr_results_pool_' respectively.
-  std::vector<AggFnEvaluator*> agg_fn_evals_;
-
   /// Exprs used to evaluate input rows
   std::vector<ScalarExpr*> grouping_exprs_;
 
@@ -235,36 +201,42 @@ class PartitionedAggregationNode : public ExecNode {
   RuntimeState* state_;
 
   /// Allocator for hash table memory.
-  boost::scoped_ptr<Suballocator> ht_allocator_;
+  std::unique_ptr<Suballocator> ht_allocator_;
 
-  /// MemPool used to allocate memory for when we don't have grouping and don't initialize
-  /// the partitioning structures, or during Close() when creating new output tuples.
-  /// For non-grouping aggregations, the ownership of the pool's memory is transferred
-  /// to the output batch on eos. The pool should not be Reset() to allow amortizing
-  /// memory allocation over a series of Reset()/Open()/GetNext()* calls.
-  boost::scoped_ptr<MemPool> singleton_tuple_pool_;
+  /// MemPool used to allocate memory during Close() when creating new output tuples. The
+  /// pool should not be Reset() to allow amortizing memory allocation over a series of
+  /// Reset()/Open()/GetNext()* calls.
+  std::unique_ptr<MemPool> tuple_pool_;
 
   /// The current partition and iterator to the next row in its hash table that we need
   /// to return in GetNext()
   Partition* output_partition_;
   HashTable::Iterator output_iterator_;
 
-  typedef Status (*ProcessBatchNoGroupingFn)(PartitionedAggregationNode*, RowBatch*);
-  /// Jitted ProcessBatchNoGrouping function pointer. Null if codegen is disabled.
-  ProcessBatchNoGroupingFn process_batch_no_grouping_fn_;
+  /// Resource information sent from the frontend.
+  const TBackendResourceProfile resource_profile_;
 
-  typedef Status (*ProcessBatchFn)(
-      PartitionedAggregationNode*, RowBatch*, TPrefetchMode::type, HashTableCtx*);
-  /// Jitted ProcessBatch function pointer. Null if codegen is disabled.
-  ProcessBatchFn process_batch_fn_;
+  ReservationManager reservation_manager_;
+  BufferPool::ClientHandle* buffer_pool_client();
 
-  typedef Status (*ProcessBatchStreamingFn)(PartitionedAggregationNode*, bool,
-      TPrefetchMode::type, RowBatch*, RowBatch*, HashTableCtx*, int[PARTITION_FANOUT]);
-  /// Jitted ProcessBatchStreaming function pointer.  Null if codegen is disabled.
-  ProcessBatchStreamingFn process_batch_streaming_fn_;
+  /// The number of rows that have been passed to AddBatch() or AddBatchStreaming().
+  int64_t num_input_rows_;
+
+  /// True if this aggregator is being executed in a subplan.
+  const bool is_in_subplan_;
 
-  /// Time spent processing the child rows
-  RuntimeProfile::Counter* build_timer_;
+  int64_t limit_; // -1: no limit
+  bool ReachedLimit() { return limit_ != -1 && num_rows_returned_ >= limit_; }
+
+  typedef Status (*AddBatchImplFn)(
+      GroupingAggregator*, RowBatch*, TPrefetchMode::type, HashTableCtx*);
+  /// Jitted AddBatchImpl function pointer. Null if codegen is disabled.
+  AddBatchImplFn add_batch_impl_fn_;
+
+  typedef Status (*AddBatchStreamingImplFn)(GroupingAggregator*, bool,
+      TPrefetchMode::type, RowBatch*, RowBatch*, HashTableCtx*, int[PARTITION_FANOUT]);
+  /// Jitted AddBatchStreamingImpl function pointer.  Null if codegen is disabled.
+  AddBatchStreamingImplFn add_batch_streaming_impl_fn_;
 
   /// Total time spent resizing hash tables.
   RuntimeProfile::Counter* ht_resize_timer_;
@@ -309,32 +281,21 @@ class PartitionedAggregationNode : public ExecNode {
   /// The estimated number of input rows from the planner.
   int64_t estimated_input_cardinality_;
 
+  TDebugOptions debug_options_;
+
   /////////////////////////////////////////
   /// BEGIN: Members that must be Reset()
 
-  /// Result of aggregation w/o GROUP BY.
-  /// Note: can be NULL even if there is no grouping if the result tuple is 0 width
-  /// e.g. select 1 from table group by col.
-  Tuple* singleton_output_tuple_;
-  bool singleton_output_tuple_returned_;
-
-  /// Row batch used as argument to GetNext() for the child node preaggregations. Store
-  /// in node to avoid reallocating for every GetNext() call when streaming.
-  boost::scoped_ptr<RowBatch> child_batch_;
-
   /// If true, no more rows to output from partitions.
   bool partition_eos_;
 
-  /// True if no more rows to process from child.
-  bool child_eos_;
-
   /// Used for hash-related functionality, such as evaluating rows and calculating hashes.
   /// It also owns the evaluators for the grouping and build expressions used during hash
   /// table insertion and probing.
   boost::scoped_ptr<HashTableCtx> ht_ctx_;
 
   /// Object pool that holds the Partition objects in hash_partitions_.
-  boost::scoped_ptr<ObjectPool> partition_pool_;
+  std::unique_ptr<ObjectPool> partition_pool_;
 
   /// Current partitions we are partitioning into. IMPALA-5788: For the case where we
   /// rebuild a spilled partition that fits in memory, all pointers in this vector will
@@ -362,7 +323,7 @@ class PartitionedAggregationNode : public ExecNode {
   /// initially use small buffers. Streaming pre-aggregations do not spill and do not
   /// require an unaggregated stream.
   struct Partition {
-    Partition(PartitionedAggregationNode* parent, int level, int idx)
+    Partition(GroupingAggregator* parent, int level, int idx)
       : parent(parent), is_closed(false), level(level), idx(idx) {}
 
     ~Partition();
@@ -380,7 +341,7 @@ class PartitionedAggregationNode : public ExecNode {
     Status InitHashTable(bool* got_memory) WARN_UNUSED_RESULT;
 
     /// Called in case we need to serialize aggregated rows. This step effectively does
-    /// a merge aggregation in this node.
+    /// a merge aggregation in this aggregator.
     Status SerializeStreamForSpilling() WARN_UNUSED_RESULT;
 
     /// Closes this partition. If finalize_rows is true, this iterates over all rows
@@ -394,16 +355,15 @@ class PartitionedAggregationNode : public ExecNode {
     /// if 'more_aggregate_rows' is true or the unaggregated stream otherwise.
     Status Spill(bool more_aggregate_rows) WARN_UNUSED_RESULT;
 
-    bool is_spilled() const { return hash_tbl.get() == NULL; }
+    bool is_spilled() const { return hash_tbl.get() == nullptr; }
 
-    PartitionedAggregationNode* parent;
+    GroupingAggregator* parent;
 
     /// If true, this partition is closed and there is nothing left to do.
     bool is_closed;
 
     /// How many times rows in this partition have been repartitioned. Partitions created
-    /// from the node's children's input is level 0, 1 after the first repartitionining,
-    /// etc.
+    /// from the aggregator's input is level 0, 1 after the first repartitionining, etc.
     const int level;
 
     /// The index of this partition within 'hash_partitions_' at its level.
@@ -412,17 +372,16 @@ class PartitionedAggregationNode : public ExecNode {
     /// Hash table for this partition.
     /// Can be NULL if this partition is no longer maintaining a hash table (i.e.
     /// is spilled or we are passing through all rows for this partition).
-    boost::scoped_ptr<HashTable> hash_tbl;
+    std::unique_ptr<HashTable> hash_tbl;
 
     /// Clone of parent's agg_fn_evals_. Permanent allocations come from
-    /// 'agg_fn_perm_pool' and result allocations come from the ExecNode's
-    /// 'expr_results_pool_'.
+    /// 'agg_fn_perm_pool' and result allocations come from 'expr_results_pool_'.
     std::vector<AggFnEvaluator*> agg_fn_evals;
 
     /// Pool for permanent allocations for this partition's 'agg_fn_evals'. Freed at the
     /// same times as 'agg_fn_evals' are closed: either when the partition is closed or
     /// when it is spilled.
-    boost::scoped_ptr<MemPool> agg_fn_perm_pool;
+    std::unique_ptr<MemPool> agg_fn_perm_pool;
 
     /// Tuple stream used to store aggregated rows. When the partition is not spilled,
     /// (meaning the hash table is maintained), this stream is pinned and contains the
@@ -432,18 +391,18 @@ class PartitionedAggregationNode : public ExecNode {
     /// For streaming preaggs, this may be NULL if sufficient memory is not available.
     /// In that case hash_tbl is also NULL and all rows for the partition will be passed
     /// through.
-    boost::scoped_ptr<BufferedTupleStream> aggregated_row_stream;
+    std::unique_ptr<BufferedTupleStream> aggregated_row_stream;
 
     /// Unaggregated rows that are spilled. Always NULL for streaming pre-aggregations.
     /// Always unpinned. Has a write buffer allocated when the partition is spilled and
     /// unaggregated rows are being processed.
-    boost::scoped_ptr<BufferedTupleStream> unaggregated_row_stream;
+    std::unique_ptr<BufferedTupleStream> unaggregated_row_stream;
   };
 
   /// Stream used to store serialized spilled rows. Only used if needs_serialize_
   /// is set. This stream is never pinned and only used in Partition::Spill as a
   /// a temporary buffer.
-  boost::scoped_ptr<BufferedTupleStream> serialize_stream_;
+  std::unique_ptr<BufferedTupleStream> serialize_stream_;
 
   /// Accessor for 'hash_tbls_' that verifies consistency with the partitions.
   HashTable* ALWAYS_INLINE GetHashTable(int partition_idx) {
@@ -452,10 +411,6 @@ class PartitionedAggregationNode : public ExecNode {
     return ht;
   }
 
-  /// Constructs singleton output tuple, allocating memory from pool.
-  Tuple* ConstructSingletonOutputTuple(
-      const std::vector<AggFnEvaluator*>& agg_fn_evals, MemPool* pool);
-
   /// Copies grouping values stored in 'ht_ctx_' that were computed over 'current_row_'
   /// using 'grouping_expr_evals_'. Aggregation expr slots are set to their initial
   /// values. Returns NULL if there was not enough memory to allocate the tuple or errors
@@ -481,39 +436,6 @@ class PartitionedAggregationNode : public ExecNode {
   /// size of the variable-length data: 'varlen_size'.
   void CopyGroupingValues(Tuple* intermediate_tuple, uint8_t* buffer, int varlen_size);
 
-  /// Initializes the aggregate function slots of an intermediate tuple.
-  /// Any var-len data is allocated from the FunctionContexts.
-  void InitAggSlots(const std::vector<AggFnEvaluator*>& agg_fn_evals,
-      Tuple* intermediate_tuple);
-
-  /// Updates the given aggregation intermediate tuple with aggregation values computed
-  /// over 'row' using 'agg_fn_evals'. Whether the agg fn evaluator calls Update() or
-  /// Merge() is controlled by the evaluator itself, unless enforced explicitly by passing
-  /// in is_merge == true.  The override is needed to merge spilled and non-spilled rows
-  /// belonging to the same partition independent of whether the agg fn evaluators have
-  /// is_merge() == true.
-  /// This function is replaced by codegen (which is why we don't use a vector argument
-  /// for agg_fn_evals).. Any var-len data is allocated from the FunctionContexts.
-  /// TODO: Fix the arguments order. Need to update CodegenUpdateTuple() too.
-  void UpdateTuple(AggFnEvaluator** agg_fn_evals, Tuple* tuple, TupleRow* row,
-      bool is_merge = false) noexcept;
-
-  /// Called on the intermediate tuple of each group after all input rows have been
-  /// consumed and aggregated. Computes the final aggregate values to be returned in
-  /// GetNext() using the agg fn evaluators' Serialize() or Finalize().
-  /// For the Finalize() case if the output tuple is different from the intermediate
-  /// tuple, then a new tuple is allocated from 'pool' to hold the final result.
-  /// Grouping values are copied into the output tuple and the the output tuple holding
-  /// the finalized/serialized aggregate values is returned.
-  /// TODO: Coordinate the allocation of new tuples with the release of memory
-  /// so as not to make memory consumption blow up.
-  Tuple* GetOutputTuple(const std::vector<AggFnEvaluator*>& agg_fn_evals,
-      Tuple* tuple, MemPool* pool);
-
-  /// Do the aggregation for all tuple rows in the batch when there is no grouping.
-  /// This function is replaced by codegen.
-  Status ProcessBatchNoGrouping(RowBatch* batch) WARN_UNUSED_RESULT;
-
   /// Processes a batch of rows. This is the core function of the algorithm. We partition
   /// the rows into hash_partitions_, spilling as necessary.
   /// If AGGREGATED_ROWS is true, it means that the rows in the batch are already
@@ -525,7 +447,7 @@ class PartitionedAggregationNode : public ExecNode {
   /// This function is replaced by codegen. We pass in ht_ctx_.get() as an argument for
   /// performance.
   template <bool AGGREGATED_ROWS>
-  Status IR_ALWAYS_INLINE ProcessBatch(RowBatch* batch, TPrefetchMode::type prefetch_mode,
+  Status IR_ALWAYS_INLINE AddBatchImpl(RowBatch* batch, TPrefetchMode::type prefetch_mode,
       HashTableCtx* ht_ctx) WARN_UNUSED_RESULT;
 
   /// Evaluates the rows in 'batch' starting at 'start_row_idx' and stores the results in
@@ -533,12 +455,12 @@ class PartitionedAggregationNode : public ExecNode {
   /// the capacity of the cache. 'prefetch_mode' specifies the prefetching mode in use.
   /// If it's not PREFETCH_NONE, hash table buckets for the computed hashes will be
   /// prefetched. Note that codegen replaces 'prefetch_mode' with a constant.
-  template<bool AGGREGATED_ROWS>
+  template <bool AGGREGATED_ROWS>
   void EvalAndHashPrefetchGroup(RowBatch* batch, int start_row_idx,
       TPrefetchMode::type prefetch_mode, HashTableCtx* ht_ctx);
 
-  /// This function processes each individual row in ProcessBatch(). Must be inlined into
-  /// ProcessBatch for codegen to substitute function calls with codegen'd versions.
+  /// This function processes each individual row in AddBatchImpl(). Must be inlined into
+  /// AddBatchImpl for codegen to substitute function calls with codegen'd versions.
   /// May spill partitions if not enough memory is available.
   template <bool AGGREGATED_ROWS>
   Status IR_ALWAYS_INLINE ProcessRow(
@@ -548,7 +470,7 @@ class PartitionedAggregationNode : public ExecNode {
   /// the context for the partition's hash table and hash is the precomputed hash of
   /// the row. The row can be an unaggregated or aggregated row depending on
   /// AGGREGATED_ROWS. Spills partitions if necessary to append the new intermediate
-  /// tuple to the partition's stream. Must be inlined into ProcessBatch for codegen
+  /// tuple to the partition's stream. Must be inlined into AddBatchImpl for codegen
   /// to substitute function calls with codegen'd versions.  insert_it is an iterator
   /// for insertion returned from HashTable::FindBuildRowBucket().
   template <bool AGGREGATED_ROWS>
@@ -562,23 +484,15 @@ class PartitionedAggregationNode : public ExecNode {
   Status IR_ALWAYS_INLINE AppendSpilledRow(
       Partition* partition, TupleRow* row) WARN_UNUSED_RESULT;
 
-  /// Reads all the rows from input_stream and process them by calling ProcessBatch().
+  /// Reads all the rows from input_stream and process them by calling AddBatchImpl().
   template <bool AGGREGATED_ROWS>
   Status ProcessStream(BufferedTupleStream* input_stream) WARN_UNUSED_RESULT;
 
-  /// Output 'singleton_output_tuple_' and transfer memory to 'row_batch'.
-  void GetSingletonOutput(RowBatch* row_batch);
-
   /// Get rows for the next rowbatch from the next partition. Sets 'partition_eos_' to
   /// true if all rows from all partitions have been returned or the limit is reached.
   Status GetRowsFromPartition(
       RuntimeState* state, RowBatch* row_batch) WARN_UNUSED_RESULT;
 
-  /// Get output rows from child for streaming pre-aggregation. Aggregates some rows with
-  /// hash table and passes through other rows converted into the intermediate
-  /// tuple format. Sets 'child_eos_' once all rows from child have been returned.
-  Status GetRowsStreaming(RuntimeState* state, RowBatch* row_batch) WARN_UNUSED_RESULT;
-
   /// Return true if we should keep expanding hash tables in the preagg. If false,
   /// the preagg should pass through any rows it can't fit in its tables.
   bool ShouldExpandPreaggHashTables() const;
@@ -594,9 +508,9 @@ class PartitionedAggregationNode : public ExecNode {
   ///     that 'prefetch_mode' will be substituted with constants during codegen time.
   /// 'remaining_capacity' is an array with PARTITION_FANOUT entries with the number of
   ///     additional rows that can be added to the hash table per partition. It is updated
-  ///     by ProcessBatchStreaming() when it inserts new rows.
+  ///     by AddBatchStreamingImpl() when it inserts new rows.
   /// 'ht_ctx' is passed in as a way to avoid aliasing of 'this' confusing the optimiser.
-  Status ProcessBatchStreaming(bool needs_serialize, TPrefetchMode::type prefetch_mode,
+  Status AddBatchStreamingImpl(bool needs_serialize, TPrefetchMode::type prefetch_mode,
       RowBatch* in_batch, RowBatch* out_batch, HashTableCtx* ht_ctx,
       int remaining_capacity[PARTITION_FANOUT]) WARN_UNUSED_RESULT;
 
@@ -670,40 +584,27 @@ class PartitionedAggregationNode : public ExecNode {
   void ClosePartitions();
 
   /// Calls finalizes on all tuples starting at 'it'.
-  void CleanupHashTbl(const std::vector<AggFnEvaluator*>& agg_fn_evals,
-      HashTable::Iterator it);
-
-  /// Codegen for updating aggregate expressions agg_fns_[agg_fn_idx]
-  /// and returns the IR function in 'fn'. Returns non-OK status if codegen
-  /// is unsuccessful.
-  Status CodegenUpdateSlot(LlvmCodeGen* codegen, int agg_fn_idx,
-      SlotDescriptor* slot_desc, llvm::Function** fn) WARN_UNUSED_RESULT;
-
-  /// Codegen a call to a function implementing the UDA interface with input values
-  /// from 'input_vals'. 'dst_val' should contain the previous value of the aggregate
-  /// function, and 'updated_dst_val' is set to the new value after the Update or Merge
-  /// operation is applied. The instruction sequence for the UDA call is inserted at
-  /// the insert position of 'builder'.
-  Status CodegenCallUda(LlvmCodeGen* codegen, LlvmBuilder* builder, AggFn* agg_fn,
-      llvm::Value* agg_fn_ctx_arg, const std::vector<CodegenAnyVal>& input_vals,
-      const CodegenAnyVal& dst_val, CodegenAnyVal* updated_dst_val) WARN_UNUSED_RESULT;
-
-  /// Codegen UpdateTuple(). Returns non-OK status if codegen is unsuccessful.
-  Status CodegenUpdateTuple(LlvmCodeGen* codegen, llvm::Function** fn) WARN_UNUSED_RESULT;
-
-  /// Codegen the non-streaming process row batch loop. The loop has already been
-  /// compiled to IR and loaded into the codegen object. UpdateAggTuple has also been
-  /// codegen'd to IR. This function will modify the loop subsituting the statically
-  /// compiled functions with codegen'd ones. 'process_batch_fn_' or
-  /// 'process_batch_no_grouping_fn_' will be updated with the codegened function
-  /// depending on whether this is a grouping or non-grouping aggregation.
+  void CleanupHashTbl(
+      const std::vector<AggFnEvaluator*>& agg_fn_evals, HashTable::Iterator it);
+
+  /// Clears 'expr_results_pool_' and returns the result of state->CheckQueryState().
+  /// Aggregators should call this periodically, e.g. once per input row batch. This
+  /// should not be called outside the main execution thread.
+  /// TODO: IMPALA-2399: replace QueryMaintenance() - see JIRA for more details.
+  Status QueryMaintenance(RuntimeState* state) WARN_UNUSED_RESULT;
+
+  /// Codegen the non-streaming add row batch loop. The loop has already been compiled to
+  /// IR and loaded into the codegen object. UpdateAggTuple has also been codegen'd to IR.
+  /// This function will modify the loop subsituting the statically compiled functions
+  /// with codegen'd ones. 'add_batch_impl_fn_' will be updated with the codegened
+  // function.
   /// Assumes AGGREGATED_ROWS = false.
-  Status CodegenProcessBatch(
+  Status CodegenAddBatchImpl(
       LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) WARN_UNUSED_RESULT;
 
   /// Codegen the materialization loop for streaming preaggregations.
-  /// 'process_batch_streaming_fn_' will be updated with the codegened function.
-  Status CodegenProcessBatchStreaming(
+  /// 'add_batch_streaming_impl_fn_' will be updated with the codegened function.
+  Status CodegenAddBatchStreamingImpl(
       LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) WARN_UNUSED_RESULT;
 
   /// Compute minimum buffer reservation for grouping aggregations.
@@ -716,19 +617,8 @@ class PartitionedAggregationNode : public ExecNode {
   /// If we need to serialize, we need an additional buffer while spilling a partition
   /// as the partitions aggregate stream needs to be serialized and rewritten.
   /// We do not spill streaming preaggregations, so we do not need to reserve any buffers.
-  int64_t MinReservation() const {
-    DCHECK(!grouping_exprs_.empty());
-    // Must be kept in sync with AggregationNode.computeNodeResourceProfile() in fe.
-    if (is_streaming_preagg_) {
-      // Reserve at least one buffer and a 64kb hash table per partition.
-      return (resource_profile_.spillable_buffer_size + 64 * 1024) * PARTITION_FANOUT;
-    }
-    int num_buffers = PARTITION_FANOUT + 1 + (needs_serialize_ ? 1 : 0);
-    // Two of the buffers must fit the maximum row.
-    return resource_profile_.spillable_buffer_size * (num_buffers - 2) +
-          resource_profile_.max_row_buffer_size * 2;
-  }
+  int64_t MinReservation() const;
 };
-}
+} // namespace impala
 
-#endif
+#endif // IMPALA_EXEC_GROUPING_AGGREGATOR_H
diff --git a/be/src/exec/non-grouping-aggregator-ir.cc b/be/src/exec/non-grouping-aggregator-ir.cc
new file mode 100644
index 0000000..335cbc7
--- /dev/null
+++ b/be/src/exec/non-grouping-aggregator-ir.cc
@@ -0,0 +1,30 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/non-grouping-aggregator.h"
+
+#include "runtime/row-batch.h"
+
+using namespace impala;
+
+Status NonGroupingAggregator::AddBatchImpl(RowBatch* batch) {
+  Tuple* output_tuple = singleton_output_tuple_;
+  FOREACH_ROW(batch, 0, batch_iter) {
+    UpdateTuple(agg_fn_evals_.data(), output_tuple, batch_iter.Get());
+  }
+  return Status::OK();
+}
diff --git a/be/src/exec/non-grouping-aggregator.cc b/be/src/exec/non-grouping-aggregator.cc
new file mode 100644
index 0000000..585c264
--- /dev/null
+++ b/be/src/exec/non-grouping-aggregator.cc
@@ -0,0 +1,174 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/non-grouping-aggregator.h"
+
+#include <sstream>
+
+#include "codegen/llvm-codegen.h"
+#include "exec/exec-node.h"
+#include "exprs/agg-fn-evaluator.h"
+#include "gutil/strings/substitute.h"
+#include "runtime/descriptors.h"
+#include "runtime/mem-pool.h"
+#include "runtime/row-batch.h"
+#include "runtime/runtime-state.h"
+#include "runtime/tuple-row.h"
+#include "runtime/tuple.h"
+#include "util/runtime-profile-counters.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+NonGroupingAggregator::NonGroupingAggregator(ExecNode* exec_node, ObjectPool* pool,
+    const TPlanNode& tnode, const DescriptorTbl& descs)
+  : Aggregator(exec_node, pool, tnode, descs, "NonGroupingAggregator"),
+    add_batch_impl_fn_(nullptr),
+    singleton_output_tuple_(nullptr),
+    singleton_output_tuple_returned_(true) {}
+
+Status NonGroupingAggregator::Prepare(RuntimeState* state) {
+  RETURN_IF_ERROR(Aggregator::Prepare(state));
+  singleton_tuple_pool_.reset(new MemPool(mem_tracker_.get()));
+  return Status::OK();
+}
+
+void NonGroupingAggregator::Codegen(RuntimeState* state) {
+  LlvmCodeGen* codegen = state->codegen();
+  DCHECK(codegen != nullptr);
+  TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
+  Status codegen_status = CodegenAddBatchImpl(codegen, prefetch_mode);
+  runtime_profile()->AddCodegenMsg(codegen_status.ok(), codegen_status);
+}
+
+Status NonGroupingAggregator::Open(RuntimeState* state) {
+  RETURN_IF_ERROR(Aggregator::Open(state));
+
+  // Create the single output tuple for this non-grouping agg. This must happen after
+  // opening the aggregate evaluators.
+  singleton_output_tuple_ =
+      ConstructSingletonOutputTuple(agg_fn_evals_, singleton_tuple_pool_.get());
+  // Check for failures during AggFnEvaluator::Init().
+  RETURN_IF_ERROR(state->GetQueryStatus());
+  singleton_output_tuple_returned_ = false;
+
+  return Status::OK();
+}
+
+Status NonGroupingAggregator::GetNext(
+    RuntimeState* state, RowBatch* row_batch, bool* eos) {
+  // There was no grouping, so evaluate the conjuncts and return the single result row.
+  // We allow calling GetNext() after eos, so don't return this row again.
+  if (!singleton_output_tuple_returned_) GetSingletonOutput(row_batch);
+  singleton_output_tuple_returned_ = true;
+  *eos = true;
+  return Status::OK();
+}
+
+void NonGroupingAggregator::GetSingletonOutput(RowBatch* row_batch) {
+  int row_idx = row_batch->AddRow();
+  TupleRow* row = row_batch->GetRow(row_idx);
+  // The output row batch may reference memory allocated by Serialize() or Finalize(),
+  // allocating that memory directly from the row batch's pool means we can safely return
+  // the batch.
+  vector<ScopedResultsPool> allocate_from_batch_pool =
+      ScopedResultsPool::Create(agg_fn_evals_, row_batch->tuple_data_pool());
+  Tuple* output_tuple = GetOutputTuple(
+      agg_fn_evals_, singleton_output_tuple_, row_batch->tuple_data_pool());
+  row->SetTuple(0, output_tuple);
+  if (ExecNode::EvalConjuncts(conjunct_evals_.data(), conjunct_evals_.size(), row)) {
+    row_batch->CommitLastRow();
+    ++num_rows_returned_;
+    COUNTER_SET(rows_returned_counter_, num_rows_returned_);
+  }
+  // Keep the current chunk to amortize the memory allocation over a series
+  // of Reset()/Open()/GetNext()* calls.
+  row_batch->tuple_data_pool()->AcquireData(singleton_tuple_pool_.get(), true);
+  // This node no longer owns the memory for singleton_output_tuple_.
+  singleton_output_tuple_ = nullptr;
+}
+
+void NonGroupingAggregator::Close(RuntimeState* state) {
+  if (!singleton_output_tuple_returned_) {
+    GetOutputTuple(agg_fn_evals_, singleton_output_tuple_, singleton_tuple_pool_.get());
+  }
+
+  if (singleton_tuple_pool_.get() != nullptr) singleton_tuple_pool_->FreeAll();
+  // Must be called after singleton_tuple_pool_ is freed, so that mem_tracker_ can be
+  // closed.
+  Aggregator::Close(state);
+}
+
+Status NonGroupingAggregator::AddBatch(RuntimeState* state, RowBatch* batch) {
+  SCOPED_TIMER(build_timer_);
+
+  if (add_batch_impl_fn_ != nullptr) {
+    RETURN_IF_ERROR(add_batch_impl_fn_(this, batch));
+  } else {
+    RETURN_IF_ERROR(AddBatchImpl(batch));
+  }
+
+  return Status::OK();
+}
+
+Tuple* NonGroupingAggregator::ConstructSingletonOutputTuple(
+    const vector<AggFnEvaluator*>& agg_fn_evals, MemPool* pool) {
+  Tuple* output_tuple = Tuple::Create(intermediate_tuple_desc_->byte_size(), pool);
+  InitAggSlots(agg_fn_evals, output_tuple);
+  return output_tuple;
+}
+
+string NonGroupingAggregator::DebugString(int indentation_level) const {
+  stringstream ss;
+  DebugString(indentation_level, &ss);
+  return ss.str();
+}
+
+void NonGroupingAggregator::DebugString(int indentation_level, stringstream* out) const {
+  *out << string(indentation_level * 2, ' ');
+  *out << "NonGroupingAggregator("
+       << "intermediate_tuple_id=" << intermediate_tuple_id_
+       << " output_tuple_id=" << output_tuple_id_ << " needs_finalize=" << needs_finalize_
+       << " agg_exprs=" << AggFn::DebugString(agg_fns_);
+  *out << ")";
+}
+
+Status NonGroupingAggregator::CodegenAddBatchImpl(
+    LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) {
+  llvm::Function* update_tuple_fn;
+  RETURN_IF_ERROR(CodegenUpdateTuple(codegen, &update_tuple_fn));
+
+  // Get the cross compiled update row batch function
+  IRFunction::Type ir_fn = IRFunction::NON_GROUPING_AGG_ADD_BATCH_IMPL;
+  llvm::Function* add_batch_impl_fn = codegen->GetFunction(ir_fn, true);
+  DCHECK(add_batch_impl_fn != nullptr);
+
+  int replaced;
+  replaced = codegen->ReplaceCallSites(add_batch_impl_fn, update_tuple_fn, "UpdateTuple");
+  DCHECK_GE(replaced, 1);
+  add_batch_impl_fn = codegen->FinalizeFunction(add_batch_impl_fn);
+  if (add_batch_impl_fn == nullptr) {
+    return Status("NonGroupingAggregator::CodegenAddBatchImpl(): codegen'd "
+                  "AddBatchImpl() function failed verification, see log");
+  }
+
+  void** codegened_fn_ptr = reinterpret_cast<void**>(&add_batch_impl_fn_);
+  codegen->AddFunctionToJit(add_batch_impl_fn, codegened_fn_ptr);
+  return Status::OK();
+}
+} // namespace impala
diff --git a/be/src/exec/non-grouping-aggregator.h b/be/src/exec/non-grouping-aggregator.h
new file mode 100644
index 0000000..41b3e0d
--- /dev/null
+++ b/be/src/exec/non-grouping-aggregator.h
@@ -0,0 +1,111 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef IMPALA_EXEC_NON_GROUPING_AGGREGATOR_H
+#define IMPALA_EXEC_NON_GROUPING_AGGREGATOR_H
+
+#include <memory>
+#include <vector>
+
+#include "exec/aggregator.h"
+#include "runtime/mem-pool.h"
+
+namespace impala {
+
+class AggFnEvaluator;
+class DescriptorTbl;
+class ExecNode;
+class LlvmCodeGen;
+class ObjectPool;
+class RowBatch;
+class RuntimeState;
+class TPlanNode;
+class Tuple;
+
+/// Aggregator for doing non-grouping aggregations. Input is passed to the aggregator
+/// through AddBatch(), which generates the single output row. This Aggregator does
+/// not support streaming preaggregation.
+class NonGroupingAggregator : public Aggregator {
+ public:
+  NonGroupingAggregator(ExecNode* exec_node, ObjectPool* pool, const TPlanNode& tnode,
+      const DescriptorTbl& descs);
+
+  virtual Status Prepare(RuntimeState* state) override;
+  virtual void Codegen(RuntimeState* state) override;
+  virtual Status Open(RuntimeState* state) override;
+  virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
+  virtual Status Reset(RuntimeState* state) override { return Status::OK(); }
+  virtual void Close(RuntimeState* state) override;
+
+  virtual Status AddBatch(RuntimeState* state, RowBatch* batch) override;
+  virtual Status InputDone() override { return Status::OK(); }
+
+  virtual int num_grouping_exprs() override { return 0; }
+
+  /// NonGroupingAggregator doesn't create a buffer pool client so it doesn't need the
+  /// debug options.
+  virtual void SetDebugOptions(const TDebugOptions& debug_options) override {}
+
+  virtual std::string DebugString(int indentation_level = 0) const override;
+  virtual void DebugString(int indentation_level, std::stringstream* out) const override;
+
+ private:
+  /// MemPool used to allocate memory for 'singleton_output_tuple_'. The ownership of the
+  /// pool's memory is transferred to the output batch on eos. The pool should not be
+  /// Reset() to allow amortizing memory allocation over a series of
+  /// Reset()/Open()/GetNext()* calls.
+  std::unique_ptr<MemPool> singleton_tuple_pool_;
+
+  typedef Status (*AddBatchImplFn)(NonGroupingAggregator*, RowBatch*);
+  /// Jitted AddBatchImpl function pointer. Null if codegen is disabled.
+  AddBatchImplFn add_batch_impl_fn_;
+
+  /////////////////////////////////////////
+  /// BEGIN: Members that must be Reset()
+
+  /// Result of aggregation w/o GROUP BY.
+  /// Note: can be NULL even if there is no grouping if the result tuple is 0 width
+  /// e.g. select 1 from table group by col.
+  Tuple* singleton_output_tuple_;
+  bool singleton_output_tuple_returned_;
+
+  /// END: Members that must be Reset()
+  /////////////////////////////////////////
+
+  /// Constructs singleton output tuple, allocating memory from pool.
+  Tuple* ConstructSingletonOutputTuple(
+      const std::vector<AggFnEvaluator*>& agg_fn_evals, MemPool* pool);
+
+  /// Do the aggregation for all tuple rows in the batch when there is no grouping.
+  /// This function is replaced by codegen.
+  Status AddBatchImpl(RowBatch* batch) WARN_UNUSED_RESULT;
+
+  /// Output 'singleton_output_tuple_' and transfer memory to 'row_batch'.
+  void GetSingletonOutput(RowBatch* row_batch);
+
+  /// Codegen the non-streaming add row batch loop. The loop has already been compiled to
+  /// IR and loaded into the codegen object. UpdateAggTuple has also been codegen'd to IR.
+  /// This function will modify the loop subsituting the statically compiled functions
+  /// with codegen'd ones. 'add_batch_impl_fn_' will be updated with the codegened
+  /// function.
+  /// Assumes AGGREGATED_ROWS = false.
+  Status CodegenAddBatchImpl(
+      LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) WARN_UNUSED_RESULT;
+};
+} // namespace impala
+
+#endif // IMPALA_EXEC_NON_GROUPING_AGGREGATOR_H
diff --git a/be/src/exec/partitioned-aggregation-node.cc b/be/src/exec/partitioned-aggregation-node.cc
deleted file mode 100644
index b6b1752..0000000
--- a/be/src/exec/partitioned-aggregation-node.cc
+++ /dev/null
@@ -1,1955 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-#include "exec/partitioned-aggregation-node.h"
-
-#include <math.h>
-#include <algorithm>
-#include <set>
-#include <sstream>
-
-#include "codegen/codegen-anyval.h"
-#include "codegen/llvm-codegen.h"
-#include "exec/hash-table.inline.h"
-#include "exprs/agg-fn-evaluator.h"
-#include "exprs/anyval-util.h"
-#include "exprs/scalar-expr.h"
-#include "exprs/scalar-expr-evaluator.h"
-#include "exprs/slot-ref.h"
-#include "gutil/strings/substitute.h"
-#include "runtime/buffered-tuple-stream.inline.h"
-#include "runtime/descriptors.h"
-#include "runtime/exec-env.h"
-#include "runtime/mem-pool.h"
-#include "runtime/mem-tracker.h"
-#include "runtime/query-state.h"
-#include "runtime/raw-value.h"
-#include "runtime/row-batch.h"
-#include "runtime/runtime-state.h"
-#include "runtime/string-value.inline.h"
-#include "runtime/tuple-row.h"
-#include "runtime/tuple.h"
-#include "udf/udf-internal.h"
-#include "util/debug-util.h"
-#include "util/runtime-profile-counters.h"
-
-#include "gen-cpp/Exprs_types.h"
-#include "gen-cpp/PlanNodes_types.h"
-
-#include "common/names.h"
-
-using namespace impala;
-using namespace strings;
-
-namespace impala {
-
-const char* PartitionedAggregationNode::LLVM_CLASS_NAME =
-    "class.impala::PartitionedAggregationNode";
-
-/// The minimum reduction factor (input rows divided by output rows) to grow hash tables
-/// in a streaming preaggregation, given that the hash tables are currently the given
-/// size or above. The sizes roughly correspond to hash table sizes where the bucket
-/// arrays will fit in  a cache level. Intuitively, we don't want the working set of the
-/// aggregation to expand to the next level of cache unless we're reducing the input
-/// enough to outweigh the increased memory latency we'll incur for each hash table
-/// lookup.
-///
-/// Note that the current reduction achieved is not always a good estimate of the
-/// final reduction. It may be biased either way depending on the ordering of the
-/// input. If the input order is random, we will underestimate the final reduction
-/// factor because the probability of a row having the same key as a previous row
-/// increases as more input is processed.  If the input order is correlated with the
-/// key, skew may bias the estimate. If high cardinality keys appear first, we
-/// may overestimate and if low cardinality keys appear first, we underestimate.
-/// To estimate the eventual reduction achieved, we estimate the final reduction
-/// using the planner's estimated input cardinality and the assumption that input
-/// is in a random order. This means that we assume that the reduction factor will
-/// increase over time.
-struct StreamingHtMinReductionEntry {
-  // Use 'streaming_ht_min_reduction' if the total size of hash table bucket directories in
-  // bytes is greater than this threshold.
-  int min_ht_mem;
-  // The minimum reduction factor to expand the hash tables.
-  double streaming_ht_min_reduction;
-};
-
-// TODO: experimentally tune these values and also programmatically get the cache size
-// of the machine that we're running on.
-static const StreamingHtMinReductionEntry STREAMING_HT_MIN_REDUCTION[] = {
-  // Expand up to L2 cache always.
-  {0, 0.0},
-  // Expand into L3 cache if we look like we're getting some reduction.
-  {256 * 1024, 1.1},
-  // Expand into main memory if we're getting a significant reduction.
-  {2 * 1024 * 1024, 2.0},
-};
-
-static const int STREAMING_HT_MIN_REDUCTION_SIZE =
-    sizeof(STREAMING_HT_MIN_REDUCTION) / sizeof(STREAMING_HT_MIN_REDUCTION[0]);
-
-PartitionedAggregationNode::PartitionedAggregationNode(
-    ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
-  : ExecNode(pool, tnode, descs),
-    intermediate_tuple_id_(tnode.agg_node.intermediate_tuple_id),
-    intermediate_tuple_desc_(descs.GetTupleDescriptor(intermediate_tuple_id_)),
-    intermediate_row_desc_(intermediate_tuple_desc_, false),
-    output_tuple_id_(tnode.agg_node.output_tuple_id),
-    output_tuple_desc_(descs.GetTupleDescriptor(output_tuple_id_)),
-    needs_finalize_(tnode.agg_node.need_finalize),
-    is_streaming_preagg_(tnode.agg_node.use_streaming_preaggregation),
-    needs_serialize_(false),
-    output_partition_(NULL),
-    process_batch_no_grouping_fn_(NULL),
-    process_batch_fn_(NULL),
-    process_batch_streaming_fn_(NULL),
-    build_timer_(NULL),
-    ht_resize_timer_(NULL),
-    get_results_timer_(NULL),
-    num_hash_buckets_(NULL),
-    partitions_created_(NULL),
-    max_partition_level_(NULL),
-    num_row_repartitioned_(NULL),
-    num_repartitions_(NULL),
-    num_spilled_partitions_(NULL),
-    largest_partition_percent_(NULL),
-    streaming_timer_(NULL),
-    num_passthrough_rows_(NULL),
-    preagg_estimated_reduction_(NULL),
-    preagg_streaming_ht_min_reduction_(NULL),
-    estimated_input_cardinality_(tnode.agg_node.estimated_input_cardinality),
-    singleton_output_tuple_(NULL),
-    singleton_output_tuple_returned_(true),
-    partition_eos_(false),
-    child_eos_(false),
-    partition_pool_(new ObjectPool()) {
-  DCHECK_EQ(PARTITION_FANOUT, 1 << NUM_PARTITIONING_BITS);
-  if (is_streaming_preagg_) {
-    DCHECK(conjunct_evals_.empty()) << "Preaggs have no conjuncts";
-    DCHECK(!tnode.agg_node.grouping_exprs.empty()) << "Streaming preaggs do grouping";
-    DCHECK(limit_ == -1) << "Preaggs have no limits";
-  }
-}
-
-Status PartitionedAggregationNode::Init(const TPlanNode& tnode, RuntimeState* state) {
-  RETURN_IF_ERROR(ExecNode::Init(tnode, state));
-
-  DCHECK(intermediate_tuple_desc_ != nullptr);
-  DCHECK(output_tuple_desc_ != nullptr);
-  DCHECK_EQ(intermediate_tuple_desc_->slots().size(), output_tuple_desc_->slots().size());
-  const RowDescriptor& row_desc = *child(0)->row_desc();
-  RETURN_IF_ERROR(ScalarExpr::Create(tnode.agg_node.grouping_exprs, row_desc,
-      state, &grouping_exprs_));
-
-  // Construct build exprs from intermediate_row_desc_
-  for (int i = 0; i < grouping_exprs_.size(); ++i) {
-    SlotDescriptor* desc = intermediate_tuple_desc_->slots()[i];
-    DCHECK(desc->type().type == TYPE_NULL || desc->type() == grouping_exprs_[i]->type());
-    // Hack to avoid TYPE_NULL SlotRefs.
-    SlotRef* build_expr = pool_->Add(desc->type().type != TYPE_NULL ?
-        new SlotRef(desc) : new SlotRef(desc, TYPE_BOOLEAN));
-    build_exprs_.push_back(build_expr);
-    RETURN_IF_ERROR(build_expr->Init(intermediate_row_desc_, state));
-    if (build_expr->type().IsVarLenStringType()) string_grouping_exprs_.push_back(i);
-  }
-
-  int j = grouping_exprs_.size();
-  for (int i = 0; i < tnode.agg_node.aggregate_functions.size(); ++i, ++j) {
-    SlotDescriptor* intermediate_slot_desc = intermediate_tuple_desc_->slots()[j];
-    SlotDescriptor* output_slot_desc = output_tuple_desc_->slots()[j];
-    AggFn* agg_fn;
-    RETURN_IF_ERROR(AggFn::Create(tnode.agg_node.aggregate_functions[i], row_desc,
-        *intermediate_slot_desc, *output_slot_desc, state, &agg_fn));
-    agg_fns_.push_back(agg_fn);
-    needs_serialize_ |= agg_fn->SupportsSerialize();
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::Prepare(RuntimeState* state) {
-  SCOPED_TIMER(runtime_profile_->total_time_counter());
-
-  RETURN_IF_ERROR(ExecNode::Prepare(state));
-  state_ = state;
-
-  singleton_tuple_pool_.reset(new MemPool(mem_tracker()));
-
-  ht_resize_timer_ = ADD_TIMER(runtime_profile(), "HTResizeTime");
-  get_results_timer_ = ADD_TIMER(runtime_profile(), "GetResultsTime");
-  num_hash_buckets_ =
-      ADD_COUNTER(runtime_profile(), "HashBuckets", TUnit::UNIT);
-  partitions_created_ =
-      ADD_COUNTER(runtime_profile(), "PartitionsCreated", TUnit::UNIT);
-  largest_partition_percent_ =
-      runtime_profile()->AddHighWaterMarkCounter("LargestPartitionPercent", TUnit::UNIT);
-  if (is_streaming_preagg_) {
-    runtime_profile()->AppendExecOption("Streaming Preaggregation");
-    streaming_timer_ = ADD_TIMER(runtime_profile(), "StreamingTime");
-    num_passthrough_rows_ =
-        ADD_COUNTER(runtime_profile(), "RowsPassedThrough", TUnit::UNIT);
-    preagg_estimated_reduction_ = ADD_COUNTER(
-        runtime_profile(), "ReductionFactorEstimate", TUnit::DOUBLE_VALUE);
-    preagg_streaming_ht_min_reduction_ = ADD_COUNTER(
-        runtime_profile(), "ReductionFactorThresholdToExpand", TUnit::DOUBLE_VALUE);
-  } else {
-    build_timer_ = ADD_TIMER(runtime_profile(), "BuildTime");
-    num_row_repartitioned_ =
-        ADD_COUNTER(runtime_profile(), "RowsRepartitioned", TUnit::UNIT);
-    num_repartitions_ =
-        ADD_COUNTER(runtime_profile(), "NumRepartitions", TUnit::UNIT);
-    num_spilled_partitions_ =
-        ADD_COUNTER(runtime_profile(), "SpilledPartitions", TUnit::UNIT);
-    max_partition_level_ = runtime_profile()->AddHighWaterMarkCounter(
-        "MaxPartitionLevel", TUnit::UNIT);
-  }
-
-  RETURN_IF_ERROR(AggFnEvaluator::Create(agg_fns_, state, pool_, expr_perm_pool(),
-      expr_results_pool(), &agg_fn_evals_));
-
-  if (!grouping_exprs_.empty()) {
-    RETURN_IF_ERROR(HashTableCtx::Create(pool_, state, build_exprs_,
-        grouping_exprs_, true, vector<bool>(build_exprs_.size(), true),
-        state->fragment_hash_seed(), MAX_PARTITION_DEPTH, 1, expr_perm_pool(),
-        expr_results_pool(), expr_results_pool(), &ht_ctx_));
-  }
-  state->CheckAndAddCodegenDisabledMessage(runtime_profile());
-  return Status::OK();
-}
-
-void PartitionedAggregationNode::Codegen(RuntimeState* state) {
-  DCHECK(state->ShouldCodegen());
-  ExecNode::Codegen(state);
-  if (IsNodeCodegenDisabled()) return;
-
-  LlvmCodeGen* codegen = state->codegen();
-  DCHECK(codegen != NULL);
-  TPrefetchMode::type prefetch_mode = state_->query_options().prefetch_mode;
-  Status codegen_status = is_streaming_preagg_ ?
-      CodegenProcessBatchStreaming(codegen, prefetch_mode) :
-      CodegenProcessBatch(codegen, prefetch_mode);
-  runtime_profile()->AddCodegenMsg(codegen_status.ok(), codegen_status);
-}
-
-Status PartitionedAggregationNode::Open(RuntimeState* state) {
-  SCOPED_TIMER(runtime_profile_->total_time_counter());
-  // Open the child before consuming resources in this node.
-  RETURN_IF_ERROR(child(0)->Open(state));
-  RETURN_IF_ERROR(ExecNode::Open(state));
-
-  // Claim reservation after the child has been opened to reduce the peak reservation
-  // requirement.
-  if (!buffer_pool_client()->is_registered() && !grouping_exprs_.empty()) {
-    DCHECK_GE(resource_profile_.min_reservation, MinReservation());
-    RETURN_IF_ERROR(ClaimBufferReservation(state));
-  }
-
-  if (ht_ctx_.get() != nullptr) RETURN_IF_ERROR(ht_ctx_->Open(state));
-  RETURN_IF_ERROR(AggFnEvaluator::Open(agg_fn_evals_, state));
-  if (grouping_exprs_.empty()) {
-    // Create the single output tuple for this non-grouping agg. This must happen after
-    // opening the aggregate evaluators.
-    singleton_output_tuple_ =
-        ConstructSingletonOutputTuple(agg_fn_evals_, singleton_tuple_pool_.get());
-    // Check for failures during AggFnEvaluator::Init().
-    RETURN_IF_ERROR(state_->GetQueryStatus());
-    singleton_output_tuple_returned_ = false;
-  } else {
-    if (ht_allocator_ == nullptr) {
-      // Allocate 'serialize_stream_' and 'ht_allocator_' on the first Open() call.
-      ht_allocator_.reset(new Suballocator(state_->exec_env()->buffer_pool(),
-          buffer_pool_client(), resource_profile_.spillable_buffer_size));
-
-      if (!is_streaming_preagg_ && needs_serialize_) {
-        serialize_stream_.reset(new BufferedTupleStream(state, &intermediate_row_desc_,
-            buffer_pool_client(), resource_profile_.spillable_buffer_size,
-            resource_profile_.max_row_buffer_size));
-        RETURN_IF_ERROR(serialize_stream_->Init(id(), false));
-        bool got_buffer;
-        // Reserve the memory for 'serialize_stream_' so we don't need to scrounge up
-        // another buffer during spilling.
-        RETURN_IF_ERROR(serialize_stream_->PrepareForWrite(&got_buffer));
-        DCHECK(got_buffer) << "Accounted in min reservation"
-                           << buffer_pool_client()->DebugString();
-        DCHECK(serialize_stream_->has_write_iterator());
-      }
-    }
-    RETURN_IF_ERROR(CreateHashPartitions(0));
-  }
-
-  // Streaming preaggregations do all processing in GetNext().
-  if (is_streaming_preagg_) return Status::OK();
-
-  RowBatch batch(child(0)->row_desc(), state->batch_size(), mem_tracker());
-  // Read all the rows from the child and process them.
-  bool eos = false;
-  do {
-    RETURN_IF_CANCELLED(state);
-    RETURN_IF_ERROR(QueryMaintenance(state));
-    RETURN_IF_ERROR(children_[0]->GetNext(state, &batch, &eos));
-
-    if (UNLIKELY(VLOG_ROW_IS_ON)) {
-      for (int i = 0; i < batch.num_rows(); ++i) {
-        TupleRow* row = batch.GetRow(i);
-        VLOG_ROW << "input row: " << PrintRow(row, *children_[0]->row_desc());
-      }
-    }
-
-    TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
-    SCOPED_TIMER(build_timer_);
-    if (grouping_exprs_.empty()) {
-      if (process_batch_no_grouping_fn_ != NULL) {
-        RETURN_IF_ERROR(process_batch_no_grouping_fn_(this, &batch));
-      } else {
-        RETURN_IF_ERROR(ProcessBatchNoGrouping(&batch));
-      }
-    } else {
-      // There is grouping, so we will do partitioned aggregation.
-      if (process_batch_fn_ != NULL) {
-        RETURN_IF_ERROR(process_batch_fn_(this, &batch, prefetch_mode, ht_ctx_.get()));
-      } else {
-        RETURN_IF_ERROR(ProcessBatch<false>(&batch, prefetch_mode, ht_ctx_.get()));
-      }
-    }
-    batch.Reset();
-  } while (!eos);
-
-  // The child can be closed at this point in most cases because we have consumed all of
-  // the input from the child and transfered ownership of the resources we need. The
-  // exception is if we are inside a subplan expecting to call Open()/GetNext() on the
-  // child again,
-  if (!IsInSubplan()) child(0)->Close(state);
-  child_eos_ = true;
-
-  // Done consuming child(0)'s input. Move all the partitions in hash_partitions_
-  // to spilled_partitions_ or aggregated_partitions_. We'll finish the processing in
-  // GetNext().
-  if (!grouping_exprs_.empty()) {
-    RETURN_IF_ERROR(MoveHashPartitions(child(0)->rows_returned()));
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::GetNext(
-    RuntimeState* state, RowBatch* row_batch, bool* eos) {
-  SCOPED_TIMER(runtime_profile_->total_time_counter());
-  RETURN_IF_ERROR(ExecDebugAction(TExecNodePhase::GETNEXT, state));
-  RETURN_IF_CANCELLED(state);
-  RETURN_IF_ERROR(QueryMaintenance(state));
-
-  if (ReachedLimit()) {
-    *eos = true;
-    return Status::OK();
-  }
-
-  if (grouping_exprs_.empty()) {
-    // There was no grouping, so evaluate the conjuncts and return the single result row.
-    // We allow calling GetNext() after eos, so don't return this row again.
-    if (!singleton_output_tuple_returned_) GetSingletonOutput(row_batch);
-    singleton_output_tuple_returned_ = true;
-    *eos = true;
-    return Status::OK();
-  }
-
-  if (!child_eos_) {
-    // For streaming preaggregations, we process rows from the child as we go.
-    DCHECK(is_streaming_preagg_);
-    RETURN_IF_ERROR(GetRowsStreaming(state, row_batch));
-  } else if (!partition_eos_) {
-    RETURN_IF_ERROR(GetRowsFromPartition(state, row_batch));
-  }
-
-  *eos = partition_eos_ && child_eos_;
-  COUNTER_SET(rows_returned_counter_, num_rows_returned_);
-  return Status::OK();
-}
-
-void PartitionedAggregationNode::GetSingletonOutput(RowBatch* row_batch) {
-  DCHECK(grouping_exprs_.empty());
-  int row_idx = row_batch->AddRow();
-  TupleRow* row = row_batch->GetRow(row_idx);
-  // The output row batch may reference memory allocated by Serialize() or Finalize(),
-  // allocating that memory directly from the row batch's pool means we can safely return
-  // the batch.
-  vector<ScopedResultsPool> allocate_from_batch_pool =
-      ScopedResultsPool::Create(agg_fn_evals_, row_batch->tuple_data_pool());
-  Tuple* output_tuple = GetOutputTuple(agg_fn_evals_,
-      singleton_output_tuple_, row_batch->tuple_data_pool());
-  row->SetTuple(0, output_tuple);
-  if (ExecNode::EvalConjuncts(
-          conjunct_evals_.data(), conjunct_evals_.size(), row)) {
-    row_batch->CommitLastRow();
-    ++num_rows_returned_;
-    COUNTER_SET(rows_returned_counter_, num_rows_returned_);
-  }
-  // Keep the current chunk to amortize the memory allocation over a series
-  // of Reset()/Open()/GetNext()* calls.
-  row_batch->tuple_data_pool()->AcquireData(singleton_tuple_pool_.get(), true);
-  // This node no longer owns the memory for singleton_output_tuple_.
-  singleton_output_tuple_ = NULL;
-}
-
-Status PartitionedAggregationNode::GetRowsFromPartition(RuntimeState* state,
-    RowBatch* row_batch) {
-  DCHECK(!row_batch->AtCapacity());
-  if (output_iterator_.AtEnd()) {
-    // Done with this partition, move onto the next one.
-    if (output_partition_ != NULL) {
-      output_partition_->Close(false);
-      output_partition_ = NULL;
-    }
-    if (aggregated_partitions_.empty() && spilled_partitions_.empty()) {
-      // No more partitions, all done.
-      partition_eos_ = true;
-      return Status::OK();
-    }
-    // Process next partition.
-    RETURN_IF_ERROR(NextPartition());
-    DCHECK(output_partition_ != NULL);
-  }
-
-  SCOPED_TIMER(get_results_timer_);
-
-  // The output row batch may reference memory allocated by Serialize() or Finalize(),
-  // allocating that memory directly from the row batch's pool means we can safely return
-  // the batch.
-  vector<ScopedResultsPool> allocate_from_batch_pool = ScopedResultsPool::Create(
-        output_partition_->agg_fn_evals, row_batch->tuple_data_pool());
-  int count = 0;
-  const int N = BitUtil::RoundUpToPowerOfTwo(state->batch_size());
-  // Keeping returning rows from the current partition.
-  while (!output_iterator_.AtEnd() && !row_batch->AtCapacity()) {
-    // This loop can go on for a long time if the conjuncts are very selective. Do query
-    // maintenance every N iterations.
-    if ((count++ & (N - 1)) == 0) {
-      RETURN_IF_CANCELLED(state);
-      RETURN_IF_ERROR(QueryMaintenance(state));
-    }
-
-    int row_idx = row_batch->AddRow();
-    TupleRow* row = row_batch->GetRow(row_idx);
-    Tuple* intermediate_tuple = output_iterator_.GetTuple();
-    Tuple* output_tuple = GetOutputTuple(output_partition_->agg_fn_evals,
-        intermediate_tuple, row_batch->tuple_data_pool());
-    output_iterator_.Next();
-    row->SetTuple(0, output_tuple);
-    DCHECK_EQ(conjunct_evals_.size(), conjuncts_.size());
-    if (ExecNode::EvalConjuncts(conjunct_evals_.data(), conjuncts_.size(), row)) {
-      row_batch->CommitLastRow();
-      ++num_rows_returned_;
-      if (ReachedLimit()) break;
-    }
-  }
-
-  COUNTER_SET(rows_returned_counter_, num_rows_returned_);
-  partition_eos_ = ReachedLimit();
-  if (output_iterator_.AtEnd()) row_batch->MarkNeedsDeepCopy();
-
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::GetRowsStreaming(RuntimeState* state,
-    RowBatch* out_batch) {
-  DCHECK(!child_eos_);
-  DCHECK(is_streaming_preagg_);
-
-  if (child_batch_ == NULL) {
-    child_batch_.reset(new RowBatch(child(0)->row_desc(), state->batch_size(),
-        mem_tracker()));
-  }
-
-  do {
-    DCHECK_EQ(out_batch->num_rows(), 0);
-    RETURN_IF_CANCELLED(state);
-    RETURN_IF_ERROR(QueryMaintenance(state));
-
-    RETURN_IF_ERROR(child(0)->GetNext(state, child_batch_.get(), &child_eos_));
-
-    SCOPED_TIMER(streaming_timer_);
-
-    int remaining_capacity[PARTITION_FANOUT];
-    bool ht_needs_expansion = false;
-    for (int i = 0; i < PARTITION_FANOUT; ++i) {
-      HashTable* hash_tbl = GetHashTable(i);
-      remaining_capacity[i] = hash_tbl->NumInsertsBeforeResize();
-      ht_needs_expansion |= remaining_capacity[i] < child_batch_->num_rows();
-    }
-
-    // Stop expanding hash tables if we're not reducing the input sufficiently. As our
-    // hash tables expand out of each level of cache hierarchy, every hash table lookup
-    // will take longer. We also may not be able to expand hash tables because of memory
-    // pressure. In this case HashTable::CheckAndResize() will fail. In either case we
-    // should always use the remaining space in the hash table to avoid wasting memory.
-    if (ht_needs_expansion && ShouldExpandPreaggHashTables()) {
-      for (int i = 0; i < PARTITION_FANOUT; ++i) {
-        HashTable* ht = GetHashTable(i);
-        if (remaining_capacity[i] < child_batch_->num_rows()) {
-          SCOPED_TIMER(ht_resize_timer_);
-          bool resized;
-          RETURN_IF_ERROR(
-              ht->CheckAndResize(child_batch_->num_rows(), ht_ctx_.get(), &resized));
-          if (resized) {
-            remaining_capacity[i] = ht->NumInsertsBeforeResize();
-          }
-        }
-      }
-    }
-
-    TPrefetchMode::type prefetch_mode = state->query_options().prefetch_mode;
-    if (process_batch_streaming_fn_ != NULL) {
-      RETURN_IF_ERROR(process_batch_streaming_fn_(this, needs_serialize_, prefetch_mode,
-          child_batch_.get(), out_batch, ht_ctx_.get(), remaining_capacity));
-    } else {
-      RETURN_IF_ERROR(ProcessBatchStreaming(needs_serialize_, prefetch_mode,
-          child_batch_.get(), out_batch, ht_ctx_.get(), remaining_capacity));
-    }
-
-    child_batch_->Reset(); // All rows from child_batch_ were processed.
-  } while (out_batch->num_rows() == 0 && !child_eos_);
-
-  if (child_eos_) {
-    child(0)->Close(state);
-    child_batch_.reset();
-    RETURN_IF_ERROR(MoveHashPartitions(child(0)->rows_returned()));
-  }
-
-  num_rows_returned_ += out_batch->num_rows();
-  COUNTER_SET(num_passthrough_rows_, num_rows_returned_);
-  return Status::OK();
-}
-
-bool PartitionedAggregationNode::ShouldExpandPreaggHashTables() const {
-  int64_t ht_mem = 0;
-  int64_t ht_rows = 0;
-  for (int i = 0; i < PARTITION_FANOUT; ++i) {
-    HashTable* ht = hash_partitions_[i]->hash_tbl.get();
-    ht_mem += ht->CurrentMemSize();
-    ht_rows += ht->size();
-  }
-
-  // Need some rows in tables to have valid statistics.
-  if (ht_rows == 0) return true;
-
-  // Find the appropriate reduction factor in our table for the current hash table sizes.
-  int cache_level = 0;
-  while (cache_level + 1 < STREAMING_HT_MIN_REDUCTION_SIZE &&
-      ht_mem >= STREAMING_HT_MIN_REDUCTION[cache_level + 1].min_ht_mem) {
-    ++cache_level;
-  }
-
-  // Compare the number of rows in the hash table with the number of input rows that
-  // were aggregated into it. Exclude passed through rows from this calculation since
-  // they were not in hash tables.
-  const int64_t input_rows = children_[0]->rows_returned();
-  const int64_t aggregated_input_rows = input_rows - num_rows_returned_;
-  const int64_t expected_input_rows = estimated_input_cardinality_ - num_rows_returned_;
-  double current_reduction = static_cast<double>(aggregated_input_rows) / ht_rows;
-
-  // TODO: workaround for IMPALA-2490: subplan node rows_returned counter may be
-  // inaccurate, which could lead to a divide by zero below.
-  if (aggregated_input_rows <= 0) return true;
-
-  // Extrapolate the current reduction factor (r) using the formula
-  // R = 1 + (N / n) * (r - 1), where R is the reduction factor over the full input data
-  // set, N is the number of input rows, excluding passed-through rows, and n is the
-  // number of rows inserted or merged into the hash tables. This is a very rough
-  // approximation but is good enough to be useful.
-  // TODO: consider collecting more statistics to better estimate reduction.
-  double estimated_reduction = aggregated_input_rows >= expected_input_rows
-      ? current_reduction
-      : 1 + (expected_input_rows / aggregated_input_rows) * (current_reduction - 1);
-  double min_reduction =
-    STREAMING_HT_MIN_REDUCTION[cache_level].streaming_ht_min_reduction;
-
-  COUNTER_SET(preagg_estimated_reduction_, estimated_reduction);
-  COUNTER_SET(preagg_streaming_ht_min_reduction_, min_reduction);
-  return estimated_reduction > min_reduction;
-}
-
-void PartitionedAggregationNode::CleanupHashTbl(
-    const vector<AggFnEvaluator*>& agg_fn_evals, HashTable::Iterator it) {
-  if (!needs_finalize_ && !needs_serialize_) return;
-
-  // Iterate through the remaining rows in the hash table and call Serialize/Finalize on
-  // them in order to free any memory allocated by UDAs.
-  if (needs_finalize_) {
-    // Finalize() requires a dst tuple but we don't actually need the result,
-    // so allocate a single dummy tuple to avoid accumulating memory.
-    Tuple* dummy_dst = NULL;
-    dummy_dst = Tuple::Create(
-        output_tuple_desc_->byte_size(), singleton_tuple_pool_.get());
-    while (!it.AtEnd()) {
-      Tuple* tuple = it.GetTuple();
-      AggFnEvaluator::Finalize(agg_fn_evals, tuple, dummy_dst);
-      it.Next();
-      // Free any expr result allocations to prevent them accumulating excessively.
-      expr_results_pool_->Clear();
-    }
-  } else {
-    while (!it.AtEnd()) {
-      Tuple* tuple = it.GetTuple();
-      AggFnEvaluator::Serialize(agg_fn_evals, tuple);
-      it.Next();
-      // Free any expr result allocations to prevent them accumulating excessively.
-      expr_results_pool_->Clear();
-    }
-  }
-}
-
-Status PartitionedAggregationNode::Reset(RuntimeState* state) {
-  DCHECK(!is_streaming_preagg_) << "Cannot reset preaggregation";
-  if (!grouping_exprs_.empty()) {
-    child_eos_ = false;
-    partition_eos_ = false;
-    // Reset the HT and the partitions for this grouping agg.
-    ht_ctx_->set_level(0);
-    ClosePartitions();
-  }
-  return ExecNode::Reset(state);
-}
-
-void PartitionedAggregationNode::Close(RuntimeState* state) {
-  if (is_closed()) return;
-
-  if (!singleton_output_tuple_returned_) {
-    GetOutputTuple(agg_fn_evals_, singleton_output_tuple_, singleton_tuple_pool_.get());
-  }
-
-  // Iterate through the remaining rows in the hash table and call Serialize/Finalize on
-  // them in order to free any memory allocated by UDAs
-  if (output_partition_ != NULL) {
-    CleanupHashTbl(output_partition_->agg_fn_evals, output_iterator_);
-    output_partition_->Close(false);
-  }
-
-  ClosePartitions();
-
-  child_batch_.reset();
-
-  // Close all the agg-fn-evaluators
-  AggFnEvaluator::Close(agg_fn_evals_, state);
-
-  if (singleton_tuple_pool_.get() != nullptr) singleton_tuple_pool_->FreeAll();
-  if (ht_ctx_.get() != nullptr) ht_ctx_->Close(state);
-  ht_ctx_.reset();
-  if (serialize_stream_.get() != nullptr) {
-    serialize_stream_->Close(nullptr, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
-  }
-  ScalarExpr::Close(grouping_exprs_);
-  ScalarExpr::Close(build_exprs_);
-  AggFn::Close(agg_fns_);
-  ExecNode::Close(state);
-}
-
-PartitionedAggregationNode::Partition::~Partition() {
-  DCHECK(is_closed);
-}
-
-Status PartitionedAggregationNode::Partition::InitStreams() {
-  agg_fn_perm_pool.reset(new MemPool(parent->expr_mem_tracker()));
-  DCHECK_EQ(agg_fn_evals.size(), 0);
-  AggFnEvaluator::ShallowClone(parent->partition_pool_.get(), agg_fn_perm_pool.get(),
-      parent->expr_results_pool(), parent->agg_fn_evals_, &agg_fn_evals);
-  // Varlen aggregate function results are stored outside of aggregated_row_stream because
-  // BufferedTupleStream doesn't support relocating varlen data stored in the stream.
-  auto agg_slot = parent->intermediate_tuple_desc_->slots().begin() +
-      parent->grouping_exprs_.size();
-  set<SlotId> external_varlen_slots;
-  for (; agg_slot != parent->intermediate_tuple_desc_->slots().end(); ++agg_slot) {
-    if ((*agg_slot)->type().IsVarLenStringType()) {
-      external_varlen_slots.insert((*agg_slot)->id());
-    }
-  }
-
-  aggregated_row_stream.reset(
-      new BufferedTupleStream(parent->state_, &parent->intermediate_row_desc_,
-          parent->buffer_pool_client(), parent->resource_profile_.spillable_buffer_size,
-          parent->resource_profile_.max_row_buffer_size, external_varlen_slots));
-  RETURN_IF_ERROR(aggregated_row_stream->Init(parent->id(), true));
-  bool got_buffer;
-  RETURN_IF_ERROR(aggregated_row_stream->PrepareForWrite(&got_buffer));
-  DCHECK(got_buffer) << "Buffer included in reservation " << parent->id_ << "\n"
-                     << parent->buffer_pool_client()->DebugString() << "\n"
-                     << parent->DebugString(2);
-  if (!parent->is_streaming_preagg_) {
-    unaggregated_row_stream.reset(
-        new BufferedTupleStream(parent->state_, parent->child(0)->row_desc(),
-            parent->buffer_pool_client(), parent->resource_profile_.spillable_buffer_size,
-            parent->resource_profile_.max_row_buffer_size));
-    // This stream is only used to spill, no need to ever have this pinned.
-    RETURN_IF_ERROR(unaggregated_row_stream->Init(parent->id(), false));
-    // Save memory by waiting until we spill to allocate the write buffer for the
-    // unaggregated row stream.
-    DCHECK(!unaggregated_row_stream->has_write_iterator());
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::Partition::InitHashTable(bool* got_memory) {
-  DCHECK(aggregated_row_stream != nullptr);
-  DCHECK(hash_tbl == nullptr);
-  // We use the upper PARTITION_FANOUT num bits to pick the partition so only the
-  // remaining bits can be used for the hash table.
-  // TODO: we could switch to 64 bit hashes and then we don't need a max size.
-  // It might be reasonable to limit individual hash table size for other reasons
-  // though. Always start with small buffers.
-  hash_tbl.reset(HashTable::Create(parent->ht_allocator_.get(), false, 1, nullptr,
-      1L << (32 - NUM_PARTITIONING_BITS), PAGG_DEFAULT_HASH_TABLE_SZ));
-  // Please update the error message in CreateHashPartitions() if initial size of
-  // hash table changes.
-  return hash_tbl->Init(got_memory);
-}
-
-Status PartitionedAggregationNode::Partition::SerializeStreamForSpilling() {
-  DCHECK(!parent->is_streaming_preagg_);
-  if (parent->needs_serialize_) {
-    // We need to do a lot more work in this case. This step effectively does a merge
-    // aggregation in this node. We need to serialize the intermediates, spill the
-    // intermediates and then feed them into the aggregate function's merge step.
-    // This is often used when the intermediate is a string type, meaning the current
-    // (before serialization) in-memory layout is not the on-disk block layout.
-    // The disk layout does not support mutable rows. We need to rewrite the stream
-    // into the on disk format.
-    // TODO: if it happens to not be a string, we could serialize in place. This is
-    // a future optimization since it is very unlikely to have a serialize phase
-    // for those UDAs.
-    DCHECK(parent->serialize_stream_.get() != NULL);
-    DCHECK(!parent->serialize_stream_->is_pinned());
-
-    // Serialize and copy the spilled partition's stream into the new stream.
-    Status status;
-    BufferedTupleStream* new_stream = parent->serialize_stream_.get();
-    HashTable::Iterator it = hash_tbl->Begin(parent->ht_ctx_.get());
-    while (!it.AtEnd()) {
-      Tuple* tuple = it.GetTuple();
-      it.Next();
-      AggFnEvaluator::Serialize(agg_fn_evals, tuple);
-      if (UNLIKELY(!new_stream->AddRow(reinterpret_cast<TupleRow*>(&tuple), &status))) {
-        DCHECK(!status.ok()) << "Stream was unpinned - AddRow() only fails on error";
-        // Even if we can't add to new_stream, finish up processing this agg stream to make
-        // clean up easier (someone has to finalize this stream and we don't want to remember
-        // where we are).
-        parent->CleanupHashTbl(agg_fn_evals, it);
-        hash_tbl->Close();
-        hash_tbl.reset();
-        aggregated_row_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
-        return status;
-      }
-    }
-
-    aggregated_row_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
-    aggregated_row_stream.swap(parent->serialize_stream_);
-    // Recreate the serialize_stream (and reserve 1 buffer) now in preparation for
-    // when we need to spill again. We need to have this available before we need
-    // to spill to make sure it is available. This should be acquirable since we just
-    // freed at least one buffer from this partition's (old) aggregated_row_stream.
-    parent->serialize_stream_.reset(
-        new BufferedTupleStream(parent->state_, &parent->intermediate_row_desc_,
-            parent->buffer_pool_client(), parent->resource_profile_.spillable_buffer_size,
-            parent->resource_profile_.max_row_buffer_size));
-    status = parent->serialize_stream_->Init(parent->id(), false);
-    if (status.ok()) {
-      bool got_buffer;
-      status = parent->serialize_stream_->PrepareForWrite(&got_buffer);
-      DCHECK(!status.ok() || got_buffer) << "Accounted in min reservation";
-    }
-    if (!status.ok()) {
-      hash_tbl->Close();
-      hash_tbl.reset();
-      return status;
-    }
-    DCHECK(parent->serialize_stream_->has_write_iterator());
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::Partition::Spill(bool more_aggregate_rows) {
-  DCHECK(!parent->is_streaming_preagg_);
-  DCHECK(!is_closed);
-  DCHECK(!is_spilled());
-  RETURN_IF_ERROR(parent->state_->StartSpilling(parent->mem_tracker()));
-
-  RETURN_IF_ERROR(SerializeStreamForSpilling());
-
-  // Free the in-memory result data.
-  AggFnEvaluator::Close(agg_fn_evals, parent->state_);
-  agg_fn_evals.clear();
-
-  if (agg_fn_perm_pool.get() != nullptr) {
-    agg_fn_perm_pool->FreeAll();
-    agg_fn_perm_pool.reset();
-  }
-
-  hash_tbl->Close();
-  hash_tbl.reset();
-
-  // Unpin the stream to free memory, but leave a write buffer in place so we can
-  // continue appending rows to one of the streams in the partition.
-  DCHECK(aggregated_row_stream->has_write_iterator());
-  DCHECK(!unaggregated_row_stream->has_write_iterator());
-  if (more_aggregate_rows) {
-    aggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL_EXCEPT_CURRENT);
-  } else {
-    aggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL);
-    bool got_buffer;
-    RETURN_IF_ERROR(unaggregated_row_stream->PrepareForWrite(&got_buffer));
-    DCHECK(got_buffer) << "Accounted in min reservation"
-                       << parent->buffer_pool_client()->DebugString();
-  }
-
-  COUNTER_ADD(parent->num_spilled_partitions_, 1);
-  if (parent->num_spilled_partitions_->value() == 1) {
-    parent->runtime_profile()->AppendExecOption("Spilled");
-  }
-  return Status::OK();
-}
-
-void PartitionedAggregationNode::Partition::Close(bool finalize_rows) {
-  if (is_closed) return;
-  is_closed = true;
-  if (aggregated_row_stream.get() != NULL) {
-    if (finalize_rows && hash_tbl.get() != NULL) {
-      // We need to walk all the rows and Finalize them here so the UDA gets a chance
-      // to cleanup. If the hash table is gone (meaning this was spilled), the rows
-      // should have been finalized/serialized in Spill().
-      parent->CleanupHashTbl(agg_fn_evals, hash_tbl->Begin(parent->ht_ctx_.get()));
-    }
-    aggregated_row_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
-  }
-  if (hash_tbl.get() != NULL) hash_tbl->Close();
-  if (unaggregated_row_stream.get() != NULL) {
-    unaggregated_row_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
-  }
-  for (AggFnEvaluator* eval : agg_fn_evals) eval->Close(parent->state_);
-  if (agg_fn_perm_pool.get() != nullptr) agg_fn_perm_pool->FreeAll();
-}
-
-Tuple* PartitionedAggregationNode::ConstructSingletonOutputTuple(
-    const vector<AggFnEvaluator*>& agg_fn_evals, MemPool* pool) {
-  DCHECK(grouping_exprs_.empty());
-  Tuple* output_tuple = Tuple::Create(intermediate_tuple_desc_->byte_size(), pool);
-  InitAggSlots(agg_fn_evals, output_tuple);
-  return output_tuple;
-}
-
-Tuple* PartitionedAggregationNode::ConstructIntermediateTuple(
-    const vector<AggFnEvaluator*>& agg_fn_evals, MemPool* pool,
-    Status* status) noexcept {
-  const int fixed_size = intermediate_tuple_desc_->byte_size();
-  const int varlen_size = GroupingExprsVarlenSize();
-  const int tuple_data_size = fixed_size + varlen_size;
-  uint8_t* tuple_data = pool->TryAllocate(tuple_data_size);
-  if (UNLIKELY(tuple_data == NULL)) {
-    string details = Substitute("Cannot perform aggregation at node with id $0. Failed "
-        "to allocate $1 bytes for intermediate tuple.", id_, tuple_data_size);
-    *status = pool->mem_tracker()->MemLimitExceeded(state_, details, tuple_data_size);
-    return NULL;
-  }
-  memset(tuple_data, 0, fixed_size);
-  Tuple* intermediate_tuple = reinterpret_cast<Tuple*>(tuple_data);
-  uint8_t* varlen_data = tuple_data + fixed_size;
-  CopyGroupingValues(intermediate_tuple, varlen_data, varlen_size);
-  InitAggSlots(agg_fn_evals, intermediate_tuple);
-  return intermediate_tuple;
-}
-
-Tuple* PartitionedAggregationNode::ConstructIntermediateTuple(
-    const vector<AggFnEvaluator*>& agg_fn_evals, BufferedTupleStream* stream,
-    Status* status) noexcept {
-  DCHECK(stream != NULL && status != NULL);
-  // Allocate space for the entire tuple in the stream.
-  const int fixed_size = intermediate_tuple_desc_->byte_size();
-  const int varlen_size = GroupingExprsVarlenSize();
-  const int tuple_size = fixed_size + varlen_size;
-  uint8_t* tuple_data = stream->AddRowCustomBegin(tuple_size, status);
-  if (UNLIKELY(tuple_data == nullptr)) {
-    // If we failed to allocate and did not hit an error (indicated by a non-ok status),
-    // the caller of this function can try to free some space, e.g. through spilling, and
-    // re-attempt to allocate space for this row.
-    return nullptr;
-  }
-  Tuple* tuple = reinterpret_cast<Tuple*>(tuple_data);
-  tuple->Init(fixed_size);
-  uint8_t* varlen_buffer = tuple_data + fixed_size;
-  CopyGroupingValues(tuple, varlen_buffer, varlen_size);
-  InitAggSlots(agg_fn_evals, tuple);
-  stream->AddRowCustomEnd(tuple_size);
-  return tuple;
-}
-
-int PartitionedAggregationNode::GroupingExprsVarlenSize() {
-  int varlen_size = 0;
-  // TODO: The hash table could compute this as it hashes.
-  for (int expr_idx: string_grouping_exprs_) {
-    StringValue* sv = reinterpret_cast<StringValue*>(ht_ctx_->ExprValue(expr_idx));
-    // Avoid branching by multiplying length by null bit.
-    varlen_size += sv->len * !ht_ctx_->ExprValueNull(expr_idx);
-  }
-  return varlen_size;
-}
-
-// TODO: codegen this function.
-void PartitionedAggregationNode::CopyGroupingValues(Tuple* intermediate_tuple,
-    uint8_t* buffer, int varlen_size) {
-  // Copy over all grouping slots (the variable length data is copied below).
-  for (int i = 0; i < grouping_exprs_.size(); ++i) {
-    SlotDescriptor* slot_desc = intermediate_tuple_desc_->slots()[i];
-    if (ht_ctx_->ExprValueNull(i)) {
-      intermediate_tuple->SetNull(slot_desc->null_indicator_offset());
-    } else {
-      void* src = ht_ctx_->ExprValue(i);
-      void* dst = intermediate_tuple->GetSlot(slot_desc->tuple_offset());
-      memcpy(dst, src, slot_desc->slot_size());
-    }
-  }
-
-  for (int expr_idx: string_grouping_exprs_) {
-    if (ht_ctx_->ExprValueNull(expr_idx)) continue;
-
-    SlotDescriptor* slot_desc = intermediate_tuple_desc_->slots()[expr_idx];
-    // ptr and len were already copied to the fixed-len part of string value
-    StringValue* sv = reinterpret_cast<StringValue*>(
-        intermediate_tuple->GetSlot(slot_desc->tuple_offset()));
-    memcpy(buffer, sv->ptr, sv->len);
-    sv->ptr = reinterpret_cast<char*>(buffer);
-    buffer += sv->len;
-  }
-}
-
-// TODO: codegen this function.
-void PartitionedAggregationNode::InitAggSlots(
-    const vector<AggFnEvaluator*>& agg_fn_evals, Tuple* intermediate_tuple) {
-  vector<SlotDescriptor*>::const_iterator slot_desc =
-      intermediate_tuple_desc_->slots().begin() + grouping_exprs_.size();
-  for (int i = 0; i < agg_fn_evals.size(); ++i, ++slot_desc) {
-    // To minimize branching on the UpdateTuple path, initialize the result value so that
-    // the Add() UDA function can ignore the NULL bit of its destination value. E.g. for
-    // SUM(), if we initialize the destination value to 0 (with the NULL bit set), we can
-    // just start adding to the destination value (rather than repeatedly checking the
-    // destination NULL bit. The codegen'd version of UpdateSlot() exploits this to
-    // eliminate a branch per value.
-    //
-    // For boolean and numeric types, the default values are false/0, so the nullable
-    // aggregate functions SUM() and AVG() produce the correct result. For MIN()/MAX(),
-    // initialize the value to max/min possible value for the same effect.
-    AggFnEvaluator* eval = agg_fn_evals[i];
-    eval->Init(intermediate_tuple);
-
-    DCHECK(agg_fns_[i] == &(eval->agg_fn()));
-    const AggFn* agg_fn = agg_fns_[i];
-    const AggFn::AggregationOp agg_op = agg_fn->agg_op();
-    if ((agg_op == AggFn::MIN || agg_op == AggFn::MAX) &&
-        !agg_fn->intermediate_type().IsStringType() &&
-        !agg_fn->intermediate_type().IsTimestampType()) {
-      ExprValue default_value;
-      void* default_value_ptr = NULL;
-      if (agg_op == AggFn::MIN) {
-        default_value_ptr = default_value.SetToMax((*slot_desc)->type());
-      } else {
-        DCHECK_EQ(agg_op, AggFn::MAX);
-        default_value_ptr = default_value.SetToMin((*slot_desc)->type());
-      }
-      RawValue::Write(default_value_ptr, intermediate_tuple, *slot_desc, NULL);
-    }
-  }
-}
-
-void PartitionedAggregationNode::UpdateTuple(AggFnEvaluator** agg_fn_evals,
-    Tuple* tuple, TupleRow* row, bool is_merge) noexcept {
-  DCHECK(tuple != NULL || agg_fns_.empty());
-  for (int i = 0; i < agg_fns_.size(); ++i) {
-    if (is_merge) {
-      agg_fn_evals[i]->Merge(row->GetTuple(0), tuple);
-    } else {
-      agg_fn_evals[i]->Add(row, tuple);
-    }
-  }
-}
-
-Tuple* PartitionedAggregationNode::GetOutputTuple(
-    const vector<AggFnEvaluator*>& agg_fn_evals, Tuple* tuple, MemPool* pool) {
-  DCHECK(tuple != NULL || agg_fn_evals.empty()) << tuple;
-  Tuple* dst = tuple;
-  if (needs_finalize_ && intermediate_tuple_id_ != output_tuple_id_) {
-    dst = Tuple::Create(output_tuple_desc_->byte_size(), pool);
-  }
-  if (needs_finalize_) {
-    AggFnEvaluator::Finalize(agg_fn_evals, tuple, dst);
-  } else {
-    AggFnEvaluator::Serialize(agg_fn_evals, tuple);
-  }
-  // Copy grouping values from tuple to dst.
-  // TODO: Codegen this.
-  if (dst != tuple) {
-    int num_grouping_slots = grouping_exprs_.size();
-    for (int i = 0; i < num_grouping_slots; ++i) {
-      SlotDescriptor* src_slot_desc = intermediate_tuple_desc_->slots()[i];
-      SlotDescriptor* dst_slot_desc = output_tuple_desc_->slots()[i];
-      bool src_slot_null = tuple->IsNull(src_slot_desc->null_indicator_offset());
-      void* src_slot = NULL;
-      if (!src_slot_null) src_slot = tuple->GetSlot(src_slot_desc->tuple_offset());
-      RawValue::Write(src_slot, dst, dst_slot_desc, NULL);
-    }
-  }
-  return dst;
-}
-
-template <bool AGGREGATED_ROWS>
-Status PartitionedAggregationNode::AppendSpilledRow(
-    Partition* __restrict__ partition, TupleRow* __restrict__ row) {
-  DCHECK(!is_streaming_preagg_);
-  DCHECK(partition->is_spilled());
-  BufferedTupleStream* stream = AGGREGATED_ROWS ?
-      partition->aggregated_row_stream.get() :
-      partition->unaggregated_row_stream.get();
-  DCHECK(!stream->is_pinned());
-  Status status;
-  if (LIKELY(stream->AddRow(row, &status))) return Status::OK();
-  RETURN_IF_ERROR(status);
-
-  // Keep trying to free memory by spilling until we succeed or hit an error.
-  // Running out of partitions to spill is treated as an error by SpillPartition().
-  while (true) {
-    RETURN_IF_ERROR(SpillPartition(AGGREGATED_ROWS));
-    if (stream->AddRow(row, &status)) return Status::OK();
-    RETURN_IF_ERROR(status);
-  }
-}
-
-string PartitionedAggregationNode::DebugString(int indentation_level) const {
-  stringstream ss;
-  DebugString(indentation_level, &ss);
-  return ss.str();
-}
-
-void PartitionedAggregationNode::DebugString(
-    int indentation_level, stringstream* out) const {
-  *out << string(indentation_level * 2, ' ');
-  *out << "PartitionedAggregationNode("
-       << "intermediate_tuple_id=" << intermediate_tuple_id_
-       << " output_tuple_id=" << output_tuple_id_
-       << " needs_finalize=" << needs_finalize_
-       << " grouping_exprs=" << ScalarExpr::DebugString(grouping_exprs_)
-       << " agg_exprs=" << AggFn::DebugString(agg_fns_);
-  ExecNode::DebugString(indentation_level, out);
-  *out << ")";
-}
-
-Status PartitionedAggregationNode::CreateHashPartitions(
-    int level, int single_partition_idx) {
-  if (is_streaming_preagg_) DCHECK_EQ(level, 0);
-  if (UNLIKELY(level >= MAX_PARTITION_DEPTH)) {
-    return Status(
-        TErrorCode::PARTITIONED_AGG_MAX_PARTITION_DEPTH, id_, MAX_PARTITION_DEPTH);
-  }
-  ht_ctx_->set_level(level);
-
-  DCHECK(hash_partitions_.empty());
-  int num_partitions_created = 0;
-  for (int i = 0; i < PARTITION_FANOUT; ++i) {
-    hash_tbls_[i] = nullptr;
-    if (single_partition_idx == -1 || i == single_partition_idx) {
-      Partition* new_partition = partition_pool_->Add(new Partition(this, level, i));
-      ++num_partitions_created;
-      hash_partitions_.push_back(new_partition);
-      RETURN_IF_ERROR(new_partition->InitStreams());
-    } else {
-      hash_partitions_.push_back(nullptr);
-    }
-  }
-  // Now that all the streams are reserved (meaning we have enough memory to execute
-  // the algorithm), allocate the hash tables. These can fail and we can still continue.
-  for (int i = 0; i < PARTITION_FANOUT; ++i) {
-    Partition* partition = hash_partitions_[i];
-    if (partition == nullptr) continue;
-    if (partition->aggregated_row_stream == nullptr) {
-      // Failed to create the aggregated row stream - cannot create a hash table.
-      // Just continue with a NULL hash table so rows will be passed through.
-      DCHECK(is_streaming_preagg_);
-    } else {
-      bool got_memory;
-      RETURN_IF_ERROR(partition->InitHashTable(&got_memory));
-      // Spill the partition if we cannot create a hash table for a merge aggregation.
-      if (UNLIKELY(!got_memory)) {
-        DCHECK(!is_streaming_preagg_) << "Preagg reserves enough memory for hash tables";
-        // If we're repartitioning, we will be writing aggregated rows first.
-        RETURN_IF_ERROR(partition->Spill(level > 0));
-      }
-    }
-    hash_tbls_[i] = partition->hash_tbl.get();
-  }
-  // In this case we did not have to repartition, so ensure that while building the hash
-  // table all rows will be inserted into the partition at 'single_partition_idx' in case
-  // a non deterministic grouping expression causes a row to hash to a different
-  // partition index.
-  if (single_partition_idx != -1) {
-    Partition* partition = hash_partitions_[single_partition_idx];
-    for (int i = 0; i < PARTITION_FANOUT; ++i) {
-      hash_partitions_[i] = partition;
-      hash_tbls_[i] = partition->hash_tbl.get();
-    }
-  }
-
-  COUNTER_ADD(partitions_created_, num_partitions_created);
-  if (!is_streaming_preagg_) {
-    COUNTER_SET(max_partition_level_, level);
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::CheckAndResizeHashPartitions(
-    bool partitioning_aggregated_rows, int num_rows, const HashTableCtx* ht_ctx) {
-  DCHECK(!is_streaming_preagg_);
-  for (int i = 0; i < PARTITION_FANOUT; ++i) {
-    Partition* partition = hash_partitions_[i];
-    if (partition == nullptr) continue;
-    while (!partition->is_spilled()) {
-      {
-        SCOPED_TIMER(ht_resize_timer_);
-        bool resized;
-        RETURN_IF_ERROR(partition->hash_tbl->CheckAndResize(num_rows, ht_ctx, &resized));
-        if (resized) break;
-      }
-      RETURN_IF_ERROR(SpillPartition(partitioning_aggregated_rows));
-    }
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::NextPartition() {
-  DCHECK(output_partition_ == nullptr);
-
-  if (!IsInSubplan() && spilled_partitions_.empty()) {
-    // All partitions are in memory. Release reservation that was used for previous
-    // partitions that is no longer needed. If we have spilled partitions, we want to
-    // hold onto all reservation in case it is needed to process the spilled partitions.
-    DCHECK(!buffer_pool_client()->has_unpinned_pages());
-    Status status = ReleaseUnusedReservation();
-    DCHECK(status.ok()) << "Should not fail - all partitions are in memory so there are "
-                        << "no unpinned pages. " << status.GetDetail();
-  }
-
-  // Keep looping until we get to a partition that fits in memory.
-  Partition* partition = nullptr;
-  while (true) {
-    // First return partitions that are fully aggregated (and in memory).
-    if (!aggregated_partitions_.empty()) {
-      partition = aggregated_partitions_.front();
-      DCHECK(!partition->is_spilled());
-      aggregated_partitions_.pop_front();
-      break;
-    }
-
-    // No aggregated partitions in memory - we should not be using any reservation aside
-    // from 'serialize_stream_'.
-    DCHECK_EQ(serialize_stream_ != nullptr ? serialize_stream_->BytesPinned(false) : 0,
-        buffer_pool_client()->GetUsedReservation())
-        << buffer_pool_client()->DebugString();
-
-    // Try to fit a single spilled partition in memory. We can often do this because
-    // we only need to fit 1/PARTITION_FANOUT of the data in memory.
-    // TODO: in some cases when the partition probably won't fit in memory it could
-    // be better to skip directly to repartitioning.
-    RETURN_IF_ERROR(BuildSpilledPartition(&partition));
-    if (partition != nullptr) break;
-
-    // If we can't fit the partition in memory, repartition it.
-    RETURN_IF_ERROR(RepartitionSpilledPartition());
-  }
-  DCHECK(!partition->is_spilled());
-  DCHECK(partition->hash_tbl.get() != nullptr);
-  DCHECK(partition->aggregated_row_stream->is_pinned());
-
-  output_partition_ = partition;
-  output_iterator_ = output_partition_->hash_tbl->Begin(ht_ctx_.get());
-  COUNTER_ADD(num_hash_buckets_, output_partition_->hash_tbl->num_buckets());
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::BuildSpilledPartition(Partition** built_partition) {
-  DCHECK(!spilled_partitions_.empty());
-  DCHECK(!is_streaming_preagg_);
-  // Leave the partition in 'spilled_partitions_' to be closed if we hit an error.
-  Partition* src_partition = spilled_partitions_.front();
-  DCHECK(src_partition->is_spilled());
-
-  // Create a new hash partition from the rows of the spilled partition. This is simpler
-  // than trying to finish building a partially-built partition in place. We only
-  // initialise one hash partition that all rows in 'src_partition' will hash to.
-  RETURN_IF_ERROR(CreateHashPartitions(src_partition->level, src_partition->idx));
-  Partition* dst_partition = hash_partitions_[src_partition->idx];
-  DCHECK(dst_partition != nullptr);
-
-  // Rebuild the hash table over spilled aggregate rows then start adding unaggregated
-  // rows to the hash table. It's possible the partition will spill at either stage.
-  // In that case we need to finish processing 'src_partition' so that all rows are
-  // appended to 'dst_partition'.
-  // TODO: if the partition spills again but the aggregation reduces the input
-  // significantly, we could do better here by keeping the incomplete hash table in
-  // memory and only spilling unaggregated rows that didn't fit in the hash table
-  // (somewhat similar to the passthrough pre-aggregation).
-  RETURN_IF_ERROR(ProcessStream<true>(src_partition->aggregated_row_stream.get()));
-  RETURN_IF_ERROR(ProcessStream<false>(src_partition->unaggregated_row_stream.get()));
-  src_partition->Close(false);
-  spilled_partitions_.pop_front();
-  hash_partitions_.clear();
-
-  if (dst_partition->is_spilled()) {
-    PushSpilledPartition(dst_partition);
-    *built_partition = nullptr;
-    // Spilled the partition - we should not be using any reservation except from
-    // 'serialize_stream_'.
-    DCHECK_EQ(serialize_stream_ != nullptr ? serialize_stream_->BytesPinned(false) : 0,
-        buffer_pool_client()->GetUsedReservation())
-        << buffer_pool_client()->DebugString();
-  } else {
-    *built_partition = dst_partition;
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::RepartitionSpilledPartition() {
-  DCHECK(!spilled_partitions_.empty());
-  DCHECK(!is_streaming_preagg_);
-  // Leave the partition in 'spilled_partitions_' to be closed if we hit an error.
-  Partition* partition = spilled_partitions_.front();
-  DCHECK(partition->is_spilled());
-
-  // Create the new hash partitions to repartition into. This will allocate a
-  // write buffer for each partition's aggregated row stream.
-  RETURN_IF_ERROR(CreateHashPartitions(partition->level + 1));
-  COUNTER_ADD(num_repartitions_, 1);
-
-  // Rows in this partition could have been spilled into two streams, depending
-  // on if it is an aggregated intermediate, or an unaggregated row. Aggregated
-  // rows are processed first to save a hash table lookup in ProcessBatch().
-  RETURN_IF_ERROR(ProcessStream<true>(partition->aggregated_row_stream.get()));
-
-  // Prepare write buffers so we can append spilled rows to unaggregated partitions.
-  for (Partition* hash_partition : hash_partitions_) {
-    if (!hash_partition->is_spilled()) continue;
-    // The aggregated rows have been repartitioned. Free up at least a buffer's worth of
-    // reservation and use it to pin the unaggregated write buffer.
-    hash_partition->aggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL);
-    bool got_buffer;
-    RETURN_IF_ERROR(
-        hash_partition->unaggregated_row_stream->PrepareForWrite(&got_buffer));
-    DCHECK(got_buffer) << "Accounted in min reservation"
-                       << buffer_pool_client()->DebugString();
-  }
-  RETURN_IF_ERROR(ProcessStream<false>(partition->unaggregated_row_stream.get()));
-
-  COUNTER_ADD(num_row_repartitioned_, partition->aggregated_row_stream->num_rows());
-  COUNTER_ADD(num_row_repartitioned_, partition->unaggregated_row_stream->num_rows());
-
-  partition->Close(false);
-  spilled_partitions_.pop_front();
-
-  // Done processing this partition. Move the new partitions into
-  // spilled_partitions_/aggregated_partitions_.
-  int64_t num_input_rows = partition->aggregated_row_stream->num_rows()
-      + partition->unaggregated_row_stream->num_rows();
-  RETURN_IF_ERROR(MoveHashPartitions(num_input_rows));
-  return Status::OK();
-}
-
-template <bool AGGREGATED_ROWS>
-Status PartitionedAggregationNode::ProcessStream(BufferedTupleStream* input_stream) {
-  DCHECK(!is_streaming_preagg_);
-  if (input_stream->num_rows() > 0) {
-    while (true) {
-      bool got_buffer = false;
-      RETURN_IF_ERROR(input_stream->PrepareForRead(true, &got_buffer));
-      if (got_buffer) break;
-      // Did not have a buffer to read the input stream. Spill and try again.
-      RETURN_IF_ERROR(SpillPartition(AGGREGATED_ROWS));
-    }
-
-    TPrefetchMode::type prefetch_mode = state_->query_options().prefetch_mode;
-    bool eos = false;
-    const RowDescriptor* desc =
-        AGGREGATED_ROWS ? &intermediate_row_desc_ : children_[0]->row_desc();
-    RowBatch batch(desc, state_->batch_size(), mem_tracker());
-    do {
-      RETURN_IF_ERROR(input_stream->GetNext(&batch, &eos));
-      RETURN_IF_ERROR(
-          ProcessBatch<AGGREGATED_ROWS>(&batch, prefetch_mode, ht_ctx_.get()));
-      RETURN_IF_ERROR(QueryMaintenance(state_));
-      batch.Reset();
-    } while (!eos);
-  }
-  input_stream->Close(NULL, RowBatch::FlushMode::NO_FLUSH_RESOURCES);
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::SpillPartition(bool more_aggregate_rows) {
-  int64_t max_freed_mem = 0;
-  int partition_idx = -1;
-
-  // Iterate over the partitions and pick the largest partition that is not spilled.
-  for (int i = 0; i < hash_partitions_.size(); ++i) {
-    if (hash_partitions_[i] == nullptr) continue;
-    if (hash_partitions_[i]->is_closed) continue;
-    if (hash_partitions_[i]->is_spilled()) continue;
-    // Pass 'true' because we need to keep the write block pinned. See Partition::Spill().
-    int64_t mem = hash_partitions_[i]->aggregated_row_stream->BytesPinned(true);
-    mem += hash_partitions_[i]->hash_tbl->ByteSize();
-    mem += hash_partitions_[i]->agg_fn_perm_pool->total_reserved_bytes();
-    DCHECK_GT(mem, 0); // At least the hash table buckets should occupy memory.
-    if (mem > max_freed_mem) {
-      max_freed_mem = mem;
-      partition_idx = i;
-    }
-  }
-  DCHECK_NE(partition_idx, -1) << "Should have been able to spill a partition to "
-                               << "reclaim memory: "
-                               << buffer_pool_client()->DebugString();
-  // Remove references to the destroyed hash table from 'hash_tbls_'.
-  // Additionally, we might be dealing with a rebuilt spilled partition, where all
-  // partitions point to a single in-memory partition. This also ensures that 'hash_tbls_'
-  // remains consistent in that case.
-  for (int i = 0; i < PARTITION_FANOUT; ++i) {
-    if (hash_partitions_[i] == hash_partitions_[partition_idx]) hash_tbls_[i] = nullptr;
-  }
-  return hash_partitions_[partition_idx]->Spill(more_aggregate_rows);
-}
-
-Status PartitionedAggregationNode::MoveHashPartitions(int64_t num_input_rows) {
-  DCHECK(!hash_partitions_.empty());
-  stringstream ss;
-  ss << "PA(node_id=" << id() << ") partitioned(level=" << hash_partitions_[0]->level
-     << ") " << num_input_rows << " rows into:" << endl;
-  for (int i = 0; i < hash_partitions_.size(); ++i) {
-    Partition* partition = hash_partitions_[i];
-    if (partition == nullptr) continue;
-    // We might be dealing with a rebuilt spilled partition, where all partitions are
-    // pointing to a single in-memory partition, so make sure we only proceed for the
-    // right partition.
-    if(i != partition->idx) continue;
-    int64_t aggregated_rows = 0;
-    if (partition->aggregated_row_stream != nullptr) {
-      aggregated_rows = partition->aggregated_row_stream->num_rows();
-    }
-    int64_t unaggregated_rows = 0;
-    if (partition->unaggregated_row_stream != nullptr) {
-      unaggregated_rows = partition->unaggregated_row_stream->num_rows();
-    }
-    double total_rows = aggregated_rows + unaggregated_rows;
-    double percent = total_rows * 100 / num_input_rows;
-    ss << "  " << i << " "  << (partition->is_spilled() ? "spilled" : "not spilled")
-       << " (fraction=" << fixed << setprecision(2) << percent << "%)" << endl
-       << "    #aggregated rows:" << aggregated_rows << endl
-       << "    #unaggregated rows: " << unaggregated_rows << endl;
-
-    // TODO: update counters to support doubles.
-    COUNTER_SET(largest_partition_percent_, static_cast<int64_t>(percent));
-
-    if (total_rows == 0) {
-      partition->Close(false);
-    } else if (partition->is_spilled()) {
-      PushSpilledPartition(partition);
-    } else {
-      aggregated_partitions_.push_back(partition);
-    }
-  }
-  VLOG(2) << ss.str();
-  hash_partitions_.clear();
-  return Status::OK();
-}
-
-void PartitionedAggregationNode::PushSpilledPartition(Partition* partition) {
-  DCHECK(partition->is_spilled());
-  DCHECK(partition->hash_tbl == nullptr);
-  // Ensure all pages in the spilled partition's streams are unpinned by invalidating
-  // the streams' read and write iterators. We may need all the memory to process the
-  // next spilled partitions.
-  partition->aggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL);
-  partition->unaggregated_row_stream->UnpinStream(BufferedTupleStream::UNPIN_ALL);
-  spilled_partitions_.push_front(partition);
-}
-
-void PartitionedAggregationNode::ClosePartitions() {
-  for (Partition* partition : hash_partitions_) {
-    if (partition != nullptr) partition->Close(true);
-  }
-  hash_partitions_.clear();
-  for (Partition* partition : aggregated_partitions_) partition->Close(true);
-  aggregated_partitions_.clear();
-  for (Partition* partition : spilled_partitions_) partition->Close(true);
-  spilled_partitions_.clear();
-  memset(hash_tbls_, 0, sizeof(hash_tbls_));
-  partition_pool_->Clear();
-}
-
-// IR Generation for updating a single aggregation slot. Signature is:
-// void UpdateSlot(AggFnEvaluator* agg_expr_eval, AggTuple* agg_tuple, char** row)
-//
-// The IR for sum(double_col), which is constructed directly with the IRBuilder, is:
-//
-// define void @UpdateSlot(%"class.impala::AggFnEvaluator"* %agg_fn_eval,
-//     <{ double, i8 }>* %agg_tuple, %"class.impala::TupleRow"* %row) #33 {
-// entry:
-//   %input_evals_vector = call %"class.impala::ScalarExprEvaluator"**
-//       @_ZNK6impala14AggFnEvaluator11input_evalsEv(
-//           %"class.impala::AggFnEvaluator"* %agg_fn_eval)
-//   %0 = getelementptr %"class.impala::ScalarExprEvaluator"*,
-//       %"class.impala::ScalarExprEvaluator"** %input_evals_vector, i32 0
-//   %input_eval = load %"class.impala::ScalarExprEvaluator"*,
-//       %"class.impala::ScalarExprEvaluator"** %0
-//   %input0 = call { i8, double } @GetSlotRef(%"class.impala::ScalarExprEvaluator"*
-//       %input_eval, %"class.impala::TupleRow"* %row)
-//   %dst_slot_ptr = getelementptr inbounds <{ double, i8 }>,
-//       <{ double, i8 }>* %agg_tuple, i32 0, i32 0
-//   %dst_val = load double, double* %dst_slot_ptr
-//   %1 = extractvalue { i8, double } %input0, 0
-//   %is_null = trunc i8 %1 to i1
-//   br i1 %is_null, label %ret, label %not_null
-//
-// ret:                                              ; preds = %not_null, %entry
-//   ret void
-//
-// not_null:                                         ; preds = %entry
-//   %val = extractvalue { i8, double } %input0, 1
-//   %2 = fadd double %dst_val, %val
-//   %3 = bitcast <{ double, i8 }>* %agg_tuple to i8*
-//   %null_byte_ptr = getelementptr inbounds i8, i8* %3, i32 8
-//   %null_byte = load i8, i8* %null_byte_ptr
-//   %null_bit_cleared = and i8 %null_byte, -2
-//   store i8 %null_bit_cleared, i8* %null_byte_ptr
-//   store double %2, double* %dst_slot_ptr
-//   br label %ret
-// }
-//
-// The IR for ndv(timestamp_col), which uses the UDA interface, is:
-//
-// define void @UpdateSlot(%"class.impala::AggFnEvaluator"* %agg_fn_eval,
-//     <{ [1024 x i8] }>* %agg_tuple,
-//     %"class.impala::TupleRow"* %row) #39 {
-// entry:
-//   %dst_lowered_ptr = alloca { i64, i8* }
-//   %0 = alloca { i64, i64 }
-//   %input_evals_vector = call %"class.impala::ScalarExprEvaluator"**
-//       @_ZNK6impala14AggFnEvaluator11input_evalsEv(
-//           %"class.impala::AggFnEvaluator"* %agg_fn_eval)
-//   %1 = getelementptr %"class.impala::ScalarExprEvaluator"*,
-//       %"class.impala::ScalarExprEvaluator"** %input_evals_vector, i32 0
-//   %input_eval = load %"class.impala::ScalarExprEvaluator"*,
-//       %"class.impala::ScalarExprEvaluator"** %1
-//   %input0 = call { i64, i64 } @GetSlotRef(
-//       %"class.impala::ScalarExprEvaluator"* %input_eval,
-//       %"class.impala::TupleRow"* %row)
-//   %dst_slot_ptr = getelementptr inbounds <{ [1024 x i8] }>,
-//       <{ [1024 x i8] }>* %agg_tuple, i32 0, i32 0
-//   %2 = bitcast [1024 x i8]* %dst_slot_ptr to i8*
-//   %dst = insertvalue { i64, i8* } zeroinitializer, i8* %2, 1
-//   %3 = extractvalue { i64, i8* } %dst, 0
-//   %4 = and i64 %3, 4294967295
-//   %5 = or i64 %4, 4398046511104
-//   %dst1 = insertvalue { i64, i8* } %dst, i64 %5, 0
-//   %agg_fn_ctx = call %"class.impala_udf::FunctionContext"*
-//       @_ZNK6impala14AggFnEvaluator10agg_fn_ctxEv(
-//          %"class.impala::AggFnEvaluator"* %agg_fn_eval)
-//   store { i64, i64 } %input0, { i64, i64 }* %0
-//   %input_unlowered_ptr =
-//       bitcast { i64, i64 }* %0 to %"struct.impala_udf::TimestampVal"*
-//   store { i64, i8* } %dst1, { i64, i8* }* %dst_lowered_ptr
-//   %dst_unlowered_ptr =
-//       bitcast { i64, i8* }* %dst_lowered_ptr to %"struct.impala_udf::StringVal"*
-//   call void @"void impala::AggregateFunctions::HllUpdate<impala_udf::TimestampVal>"(
-//       %"class.impala_udf::FunctionContext"* %agg_fn_ctx,
-//       %"struct.impala_udf::TimestampVal"* %input_unlowered_ptr,
-//       %"struct.impala_udf::StringVal"* %dst_unlowered_ptr)
-//   %anyval_result = load { i64, i8* }, { i64, i8* }* %dst_lowered_ptr
-//   br label %ret
-//
-// ret:                                              ; preds = %entry
-//   ret void
-// }
-//
-Status PartitionedAggregationNode::CodegenUpdateSlot(LlvmCodeGen* codegen, int agg_fn_idx,
-    SlotDescriptor* slot_desc, llvm::Function** fn) {
-  llvm::PointerType* agg_fn_eval_type = codegen->GetStructPtrType<AggFnEvaluator>();
-  llvm::StructType* tuple_struct = intermediate_tuple_desc_->GetLlvmStruct(codegen);
-  if (tuple_struct == NULL) {
-    return Status("PartitionedAggregationNode::CodegenUpdateSlot(): failed to generate "
-                  "intermediate tuple desc");
-  }
-  llvm::PointerType* tuple_ptr_type = codegen->GetPtrType(tuple_struct);
-  llvm::PointerType* tuple_row_ptr_type = codegen->GetStructPtrType<TupleRow>();
-
-  LlvmCodeGen::FnPrototype prototype(codegen, "UpdateSlot", codegen->void_type());
-  prototype.AddArgument(
-      LlvmCodeGen::NamedVariable("agg_fn_eval", agg_fn_eval_type));
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("agg_tuple", tuple_ptr_type));
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("row", tuple_row_ptr_type));
-
-  LlvmBuilder builder(codegen->context());
-  llvm::Value* args[3];
-  *fn = prototype.GeneratePrototype(&builder, &args[0]);
-  llvm::Value* agg_fn_eval_arg = args[0];
-  llvm::Value* agg_tuple_arg = args[1];
-  llvm::Value* row_arg = args[2];
-
-  // Get the vector of input expressions' evaluators.
-  llvm::Value* input_evals_vector = codegen->CodegenCallFunction(&builder,
-      IRFunction::AGG_FN_EVALUATOR_INPUT_EVALUATORS, agg_fn_eval_arg,
-      "input_evals_vector");
-
-  AggFn* agg_fn = agg_fns_[agg_fn_idx];
-  const int num_inputs = agg_fn->GetNumChildren();
-  DCHECK_GE(num_inputs, 1);
-  vector<CodegenAnyVal> input_vals;
-  for (int i = 0; i < num_inputs; ++i) {
-    ScalarExpr* input_expr = agg_fn->GetChild(i);
-    llvm::Function* input_expr_fn;
-    RETURN_IF_ERROR(input_expr->GetCodegendComputeFn(codegen, &input_expr_fn));
-    DCHECK(input_expr_fn != NULL);
-
-    // Call input expr function with the matching evaluator to get src slot value.
-    llvm::Value* input_eval =
-        codegen->CodegenArrayAt(&builder, input_evals_vector, i, "input_eval");
-    string input_name = Substitute("input$0", i);
-    CodegenAnyVal input_val = CodegenAnyVal::CreateCallWrapped(codegen, &builder,
-        input_expr->type(), input_expr_fn,
-        llvm::ArrayRef<llvm::Value*>({input_eval, row_arg}), input_name.c_str());
-    input_vals.push_back(input_val);
-  }
-
-  AggFn::AggregationOp agg_op = agg_fn->agg_op();
-  const ColumnType& dst_type = agg_fn->intermediate_type();
-  bool dst_is_int_or_float_or_bool = dst_type.IsIntegerType()
-      || dst_type.IsFloatingPointType() || dst_type.IsBooleanType();
-  bool dst_is_numeric_or_bool = dst_is_int_or_float_or_bool || dst_type.IsDecimalType();
-
-  llvm::BasicBlock* ret_block = llvm::BasicBlock::Create(codegen->context(), "ret", *fn);
-
-  // Emit the code to compute 'result' and set the NULL indicator if needed. First check
-  // for special cases where we can emit a very simple instruction sequence, then fall
-  // back to the general-purpose approach of calling the cross-compiled builtin UDA.
-  CodegenAnyVal& src = input_vals[0];
-
-  // 'dst_slot_ptr' points to the slot in the aggregate tuple to update.
-  llvm::Value* dst_slot_ptr = builder.CreateStructGEP(
-      NULL, agg_tuple_arg, slot_desc->llvm_field_idx(), "dst_slot_ptr");
-  // TODO: consider moving the following codegen logic to AggFn.
-  if (agg_op == AggFn::COUNT) {
-    src.CodegenBranchIfNull(&builder, ret_block);
-    llvm::Value* dst_value = builder.CreateLoad(dst_slot_ptr, "dst_val");
-    llvm::Value* result = agg_fn->is_merge() ?
-        builder.CreateAdd(dst_value, src.GetVal(), "count_sum") :
-        builder.CreateAdd(
-            dst_value, codegen->GetI64Constant(1), "count_inc");
-    builder.CreateStore(result, dst_slot_ptr);
-    DCHECK(!slot_desc->is_nullable());
-  } else if ((agg_op == AggFn::MIN || agg_op == AggFn::MAX) && dst_is_numeric_or_bool) {
-    bool is_min = agg_op == AggFn::MIN;
-    src.CodegenBranchIfNull(&builder, ret_block);
-    codegen->CodegenMinMax(
-        &builder, slot_desc->type(), src.GetVal(), dst_slot_ptr, is_min, *fn);
-
-    // Dst may have been NULL, make sure to unset the NULL bit.
-    DCHECK(slot_desc->is_nullable());
-    slot_desc->CodegenSetNullIndicator(
-        codegen, &builder, agg_tuple_arg, codegen->false_value());
-  } else if (agg_op == AggFn::SUM && dst_is_int_or_float_or_bool) {
-    src.CodegenBranchIfNull(&builder, ret_block);
-    llvm::Value* dst_value = builder.CreateLoad(dst_slot_ptr, "dst_val");
-    llvm::Value* result = dst_type.IsFloatingPointType() ?
-        builder.CreateFAdd(dst_value, src.GetVal()) :
-        builder.CreateAdd(dst_value, src.GetVal());
-    builder.CreateStore(result, dst_slot_ptr);
-
-    if (slot_desc->is_nullable()) {
-      slot_desc->CodegenSetNullIndicator(
-          codegen, &builder, agg_tuple_arg, codegen->false_value());
-    } else {
-      // 'slot_desc' is not nullable if the aggregate function is sum_init_zero(),
-      // because the slot is initialized to be zero and the null bit is nonexistent.
-      DCHECK_EQ(agg_fn->fn_name(), "sum_init_zero");
-    }
-  } else {
-    // The remaining cases are implemented using the UDA interface.
-    // Create intermediate argument 'dst' from 'dst_value'
-    CodegenAnyVal dst = CodegenAnyVal::GetNonNullVal(codegen, &builder, dst_type, "dst");
-
-    // For a subset of builtins we generate a different code sequence that exploits two
-    // properties of the builtins. First, NULL input values can be skipped. Second, the
-    // value of the slot was initialized in the right way in InitAggSlots() (e.g. 0 for
-    // SUM) that we get the right result if UpdateSlot() pretends that the NULL bit of
-    // 'dst' is unset. Empirically this optimisation makes TPC-H Q1 5-10% faster.
-    bool special_null_handling = !agg_fn->intermediate_type().IsStringType()
-        && !agg_fn->intermediate_type().IsTimestampType()
-        && (agg_op == AggFn::MIN || agg_op == AggFn::MAX
-               || agg_op == AggFn::SUM || agg_op == AggFn::AVG
-               || agg_op == AggFn::NDV);
-    if (slot_desc->is_nullable()) {
-      if (special_null_handling) {
-        src.CodegenBranchIfNull(&builder, ret_block);
-        slot_desc->CodegenSetNullIndicator(
-            codegen, &builder, agg_tuple_arg, codegen->false_value());
-      } else {
-        dst.SetIsNull(slot_desc->CodegenIsNull(codegen, &builder, agg_tuple_arg));
-      }
-    }
-    dst.LoadFromNativePtr(dst_slot_ptr);
-
-    // Get the FunctionContext object for the AggFnEvaluator.
-    llvm::Function* get_agg_fn_ctx_fn =
-        codegen->GetFunction(IRFunction::AGG_FN_EVALUATOR_AGG_FN_CTX, false);
-    DCHECK(get_agg_fn_ctx_fn != NULL);
-    llvm::Value* agg_fn_ctx_val =
-        builder.CreateCall(get_agg_fn_ctx_fn, {agg_fn_eval_arg}, "agg_fn_ctx");
-
-    // Call the UDA to update/merge 'src' into 'dst', with the result stored in
-    // 'updated_dst_val'.
-    CodegenAnyVal updated_dst_val;
-    RETURN_IF_ERROR(CodegenCallUda(codegen, &builder, agg_fn, agg_fn_ctx_val,
-        input_vals, dst, &updated_dst_val));
-    // Copy the value back to the slot. In the FIXED_UDA_INTERMEDIATE case, the
-    // UDA function writes directly to the slot so there is nothing to copy.
-    if (dst_type.type != TYPE_FIXED_UDA_INTERMEDIATE) {
-      updated_dst_val.StoreToNativePtr(dst_slot_ptr);
-    }
-
-    if (slot_desc->is_nullable() && !special_null_handling) {
-      // Set NULL bit in the slot based on the return value.
-      llvm::Value* result_is_null = updated_dst_val.GetIsNull("result_is_null");
-      slot_desc->CodegenSetNullIndicator(
-          codegen, &builder, agg_tuple_arg, result_is_null);
-    }
-  }
-  builder.CreateBr(ret_block);
-
-  builder.SetInsertPoint(ret_block);
-  builder.CreateRetVoid();
-
-  // Avoid producing huge UpdateTuple() function after inlining - LLVM's optimiser
-  // memory/CPU usage scales super-linearly with function size.
-  // E.g. compute stats on all columns of a 1000-column table previously took 4 minutes to
-  // codegen because all the UpdateSlot() functions were inlined.
-  if (agg_fn_idx >= LlvmCodeGen::CODEGEN_INLINE_EXPRS_THRESHOLD) {
-    codegen->SetNoInline(*fn);
-  }
-
-  *fn = codegen->FinalizeFunction(*fn);
-  if (*fn == NULL) {
-    return Status("PartitionedAggregationNode::CodegenUpdateSlot(): codegen'd "
-                  "UpdateSlot() function failed verification, see log");
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::CodegenCallUda(LlvmCodeGen* codegen,
-    LlvmBuilder* builder, AggFn* agg_fn, llvm::Value* agg_fn_ctx_val,
-    const vector<CodegenAnyVal>& input_vals, const CodegenAnyVal& dst_val,
-    CodegenAnyVal* updated_dst_val) {
-  llvm::Function* uda_fn;
-  RETURN_IF_ERROR(agg_fn->CodegenUpdateOrMergeFunction(codegen, &uda_fn));
-
-  // Set up arguments for call to UDA, which are the FunctionContext*, followed by
-  // pointers to all input values, followed by a pointer to the destination value.
-  vector<llvm::Value*> uda_fn_args;
-  uda_fn_args.push_back(agg_fn_ctx_val);
-
-  // Create pointers to input args to pass to uda_fn. We must use the unlowered type,
-  // e.g. IntVal, because the UDA interface expects the values to be passed as const
-  // references to the classes.
-  DCHECK_EQ(agg_fn->GetNumChildren(), input_vals.size());
-  for (int i = 0; i < input_vals.size(); ++i) {
-    uda_fn_args.push_back(input_vals[i].GetUnloweredPtr("input_unlowered_ptr"));
-  }
-
-  // Create pointer to dst to pass to uda_fn. We must use the unlowered type for the
-  // same reason as above.
-  llvm::Value* dst_lowered_ptr = dst_val.GetLoweredPtr("dst_lowered_ptr");
-  const ColumnType& dst_type = agg_fn->intermediate_type();
-  llvm::Type* dst_unlowered_ptr_type =
-      CodegenAnyVal::GetUnloweredPtrType(codegen, dst_type);
-  llvm::Value* dst_unlowered_ptr = builder->CreateBitCast(
-      dst_lowered_ptr, dst_unlowered_ptr_type, "dst_unlowered_ptr");
-  uda_fn_args.push_back(dst_unlowered_ptr);
-
-  // Call 'uda_fn'
-  builder->CreateCall(uda_fn, uda_fn_args);
-
-  // Convert intermediate 'dst_arg' back to the native type.
-  llvm::Value* anyval_result = builder->CreateLoad(dst_lowered_ptr, "anyval_result");
-
-  *updated_dst_val = CodegenAnyVal(codegen, builder, dst_type, anyval_result);
-  return Status::OK();
-}
-
-// IR codegen for the UpdateTuple loop.  This loop is query specific and based on the
-// aggregate functions.  The function signature must match the non- codegen'd UpdateTuple
-// exactly.
-// For the query:
-// select count(*), count(int_col), sum(double_col) the IR looks like:
-//
-// define void @UpdateTuple(%"class.impala::PartitionedAggregationNode"* %this_ptr,
-//     %"class.impala::AggFnEvaluator"** %agg_fn_evals, %"class.impala::Tuple"* %tuple,
-//     %"class.impala::TupleRow"* %row, i1 %is_merge) #33 {
-// entry:
-//   %tuple1 = bitcast %"class.impala::Tuple"* %tuple to <{ i64, i64, double, i8 }>*
-//   %src_slot = getelementptr inbounds <{ i64, i64, double, i8 }>,
-//       <{ i64, i64, double, i8 }>* %tuple1, i32 0, i32 0
-//   %count_star_val = load i64, i64* %src_slot
-//   %count_star_inc = add i64 %count_star_val, 1
-//   store i64 %count_star_inc, i64* %src_slot
-//   %0 = getelementptr %"class.impala::AggFnEvaluator"*,
-//       %"class.impala::AggFnEvaluator"** %agg_fn_evals, i32 1
-//   %agg_fn_eval =
-//       load %"class.impala::AggFnEvaluator"*, %"class.impala::AggFnEvaluator"** %0
-//   call void @UpdateSlot(%"class.impala::AggFnEvaluator"* %agg_fn_eval,
-//       <{ i64, i64, double, i8 }>* %tuple1, %"class.impala::TupleRow"* %row)
-//   %1 = getelementptr %"class.impala::AggFnEvaluator"*,
-//       %"class.impala::AggFnEvaluator"** %agg_fn_evals, i32 2
-//   %agg_fn_eval2 =
-//       load %"class.impala::AggFnEvaluator"*, %"class.impala::AggFnEvaluator"** %1
-//   call void @UpdateSlot.2(%"class.impala::AggFnEvaluator"* %agg_fn_eval2,
-//       <{ i64, i64, double, i8 }>* %tuple1, %"class.impala::TupleRow"* %row)
-//   ret void
-// }
-//
-Status PartitionedAggregationNode::CodegenUpdateTuple(
-    LlvmCodeGen* codegen, llvm::Function** fn) {
-  for (const SlotDescriptor* slot_desc : intermediate_tuple_desc_->slots()) {
-    if (slot_desc->type().type == TYPE_CHAR) {
-      return Status::Expected("PartitionedAggregationNode::CodegenUpdateTuple(): cannot "
-          "codegen CHAR in aggregations");
-    }
-  }
-
-  if (intermediate_tuple_desc_->GetLlvmStruct(codegen) == NULL) {
-    return Status::Expected("PartitionedAggregationNode::CodegenUpdateTuple(): failed to"
-        " generate intermediate tuple desc");
-  }
-
-  // Get the types to match the UpdateTuple signature
-  llvm::PointerType* agg_node_ptr_type =
-      codegen->GetStructPtrType<PartitionedAggregationNode>();
-  llvm::PointerType* evals_type = codegen->GetStructPtrPtrType<AggFnEvaluator>();
-  llvm::PointerType* tuple_ptr_type = codegen->GetStructPtrType<Tuple>();
-  llvm::PointerType* tuple_row_ptr_type = codegen->GetStructPtrType<TupleRow>();
-
-  llvm::StructType* tuple_struct = intermediate_tuple_desc_->GetLlvmStruct(codegen);
-  llvm::PointerType* tuple_ptr = codegen->GetPtrType(tuple_struct);
-  LlvmCodeGen::FnPrototype prototype(codegen, "UpdateTuple", codegen->void_type());
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("this_ptr", agg_node_ptr_type));
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("agg_fn_evals", evals_type));
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("tuple", tuple_ptr_type));
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("row", tuple_row_ptr_type));
-  prototype.AddArgument(LlvmCodeGen::NamedVariable("is_merge", codegen->bool_type()));
-
-  LlvmBuilder builder(codegen->context());
-  llvm::Value* args[5];
-  *fn = prototype.GeneratePrototype(&builder, &args[0]);
-  llvm::Value* agg_fn_evals_arg = args[1];
-  llvm::Value* tuple_arg = args[2];
-  llvm::Value* row_arg = args[3];
-
-  // Cast the parameter types to the internal llvm runtime types.
-  // TODO: get rid of this by using right type in function signature
-  tuple_arg = builder.CreateBitCast(tuple_arg, tuple_ptr, "tuple");
-
-  // Loop over each expr and generate the IR for that slot.  If the expr is not
-  // count(*), generate a helper IR function to update the slot and call that.
-  int j = grouping_exprs_.size();
-  for (int i = 0; i < agg_fns_.size(); ++i, ++j) {
-    SlotDescriptor* slot_desc = intermediate_tuple_desc_->slots()[j];
-    AggFn* agg_fn = agg_fns_[i];
-    if (agg_fn->is_count_star()) {
-      // TODO: we should be able to hoist this up to the loop over the batch and just
-      // increment the slot by the number of rows in the batch.
-      int field_idx = slot_desc->llvm_field_idx();
-      llvm::Value* const_one = codegen->GetI64Constant(1);
-      llvm::Value* slot_ptr =
-          builder.CreateStructGEP(NULL, tuple_arg, field_idx, "src_slot");
-      llvm::Value* slot_loaded = builder.CreateLoad(slot_ptr, "count_star_val");
-      llvm::Value* count_inc =
-          builder.CreateAdd(slot_loaded, const_one, "count_star_inc");
-      builder.CreateStore(count_inc, slot_ptr);
-    } else {
-      llvm::Function* update_slot_fn;
-      RETURN_IF_ERROR(CodegenUpdateSlot(codegen, i, slot_desc, &update_slot_fn));
-
-      // Load agg_fn_evals_[i]
-      llvm::Value* agg_fn_eval_val =
-          codegen->CodegenArrayAt(&builder, agg_fn_evals_arg, i, "agg_fn_eval");
-
-      // Call UpdateSlot(agg_fn_evals_[i], tuple, row);
-      llvm::Value* update_slot_args[] = {agg_fn_eval_val, tuple_arg, row_arg};
-      builder.CreateCall(update_slot_fn, update_slot_args);
-    }
-  }
-  builder.CreateRetVoid();
-
-  // Avoid inlining big UpdateTuple function into outer loop - we're unlikely to get
-  // any benefit from it since the function call overhead will be amortized.
-  if (agg_fns_.size() > LlvmCodeGen::CODEGEN_INLINE_EXPR_BATCH_THRESHOLD) {
-    codegen->SetNoInline(*fn);
-  }
-
-  // CodegenProcessBatch() does the final optimizations.
-  *fn = codegen->FinalizeFunction(*fn);
-  if (*fn == NULL) {
-    return Status("PartitionedAggregationNode::CodegenUpdateTuple(): codegen'd "
-                  "UpdateTuple() function failed verification, see log");
-  }
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::CodegenProcessBatch(LlvmCodeGen* codegen,
-    TPrefetchMode::type prefetch_mode) {
-  llvm::Function* update_tuple_fn;
-  RETURN_IF_ERROR(CodegenUpdateTuple(codegen, &update_tuple_fn));
-
-  // Get the cross compiled update row batch function
-  IRFunction::Type ir_fn = (!grouping_exprs_.empty() ?
-      IRFunction::PART_AGG_NODE_PROCESS_BATCH_UNAGGREGATED :
-      IRFunction::PART_AGG_NODE_PROCESS_BATCH_NO_GROUPING);
-  llvm::Function* process_batch_fn = codegen->GetFunction(ir_fn, true);
-  DCHECK(process_batch_fn != NULL);
-
-  int replaced;
-  if (!grouping_exprs_.empty()) {
-    // Codegen for grouping using hash table
-
-    // Replace prefetch_mode with constant so branches can be optimised out.
-    llvm::Value* prefetch_mode_arg = codegen->GetArgument(process_batch_fn, 3);
-    prefetch_mode_arg->replaceAllUsesWith(codegen->GetI32Constant(prefetch_mode));
-
-    // The codegen'd ProcessBatch function is only used in Open() with level_ = 0,
-    // so don't use murmur hash
-    llvm::Function* hash_fn;
-    RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, /* use murmur */ false, &hash_fn));
-
-    // Codegen HashTable::Equals<true>
-    llvm::Function* build_equals_fn;
-    RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, true, &build_equals_fn));
-
-    // Codegen for evaluating input rows
-    llvm::Function* eval_grouping_expr_fn;
-    RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(codegen, false, &eval_grouping_expr_fn));
-
-    // Replace call sites
-    replaced = codegen->ReplaceCallSites(process_batch_fn, eval_grouping_expr_fn,
-        "EvalProbeRow");
-    DCHECK_EQ(replaced, 1);
-
-    replaced = codegen->ReplaceCallSites(process_batch_fn, hash_fn, "HashRow");
-    DCHECK_EQ(replaced, 1);
-
-    replaced = codegen->ReplaceCallSites(process_batch_fn, build_equals_fn, "Equals");
-    DCHECK_EQ(replaced, 1);
-
-    HashTableCtx::HashTableReplacedConstants replaced_constants;
-    const bool stores_duplicates = false;
-    RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(codegen, stores_duplicates, 1,
-        process_batch_fn, &replaced_constants));
-    DCHECK_GE(replaced_constants.stores_nulls, 1);
-    DCHECK_GE(replaced_constants.finds_some_nulls, 1);
-    DCHECK_GE(replaced_constants.stores_duplicates, 1);
-    DCHECK_GE(replaced_constants.stores_tuples, 1);
-    DCHECK_GE(replaced_constants.quadratic_probing, 1);
-  }
-
-  replaced = codegen->ReplaceCallSites(process_batch_fn, update_tuple_fn, "UpdateTuple");
-  DCHECK_GE(replaced, 1);
-  process_batch_fn = codegen->FinalizeFunction(process_batch_fn);
-  if (process_batch_fn == NULL) {
-    return Status("PartitionedAggregationNode::CodegenProcessBatch(): codegen'd "
-        "ProcessBatch() function failed verification, see log");
-  }
-
-  void **codegened_fn_ptr = grouping_exprs_.empty() ?
-      reinterpret_cast<void**>(&process_batch_no_grouping_fn_) :
-      reinterpret_cast<void**>(&process_batch_fn_);
-  codegen->AddFunctionToJit(process_batch_fn, codegened_fn_ptr);
-  return Status::OK();
-}
-
-Status PartitionedAggregationNode::CodegenProcessBatchStreaming(
-    LlvmCodeGen* codegen, TPrefetchMode::type prefetch_mode) {
-  DCHECK(is_streaming_preagg_);
-
-  IRFunction::Type ir_fn = IRFunction::PART_AGG_NODE_PROCESS_BATCH_STREAMING;
-  llvm::Function* process_batch_streaming_fn = codegen->GetFunction(ir_fn, true);
-  DCHECK(process_batch_streaming_fn != NULL);
-
-  // Make needs_serialize arg constant so dead code can be optimised out.
-  llvm::Value* needs_serialize_arg = codegen->GetArgument(process_batch_streaming_fn, 2);
-  needs_serialize_arg->replaceAllUsesWith(codegen->GetBoolConstant(needs_serialize_));
-
-  // Replace prefetch_mode with constant so branches can be optimised out.
-  llvm::Value* prefetch_mode_arg = codegen->GetArgument(process_batch_streaming_fn, 3);
-  prefetch_mode_arg->replaceAllUsesWith(codegen->GetI32Constant(prefetch_mode));
-
-  llvm::Function* update_tuple_fn;
-  RETURN_IF_ERROR(CodegenUpdateTuple(codegen, &update_tuple_fn));
-
-  // We only use the top-level hash function for streaming aggregations.
-  llvm::Function* hash_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenHashRow(codegen, false, &hash_fn));
-
-  // Codegen HashTable::Equals
-  llvm::Function* equals_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenEquals(codegen, true, &equals_fn));
-
-  // Codegen for evaluating input rows
-  llvm::Function* eval_grouping_expr_fn;
-  RETURN_IF_ERROR(ht_ctx_->CodegenEvalRow(codegen, false, &eval_grouping_expr_fn));
-
-  // Replace call sites
-  int replaced = codegen->ReplaceCallSites(process_batch_streaming_fn, update_tuple_fn,
-      "UpdateTuple");
-  DCHECK_EQ(replaced, 2);
-
-  replaced = codegen->ReplaceCallSites(process_batch_streaming_fn, eval_grouping_expr_fn,
-      "EvalProbeRow");
-  DCHECK_EQ(replaced, 1);
-
-  replaced = codegen->ReplaceCallSites(process_batch_streaming_fn, hash_fn, "HashRow");
-  DCHECK_EQ(replaced, 1);
-
-  replaced = codegen->ReplaceCallSites(process_batch_streaming_fn, equals_fn, "Equals");
-  DCHECK_EQ(replaced, 1);
-
-  HashTableCtx::HashTableReplacedConstants replaced_constants;
-  const bool stores_duplicates = false;
-  RETURN_IF_ERROR(ht_ctx_->ReplaceHashTableConstants(codegen, stores_duplicates, 1,
-      process_batch_streaming_fn, &replaced_constants));
-  DCHECK_GE(replaced_constants.stores_nulls, 1);
-  DCHECK_GE(replaced_constants.finds_some_nulls, 1);
-  DCHECK_GE(replaced_constants.stores_duplicates, 1);
-  DCHECK_GE(replaced_constants.stores_tuples, 1);
-  DCHECK_GE(replaced_constants.quadratic_probing, 1);
-
-  DCHECK(process_batch_streaming_fn != NULL);
-  process_batch_streaming_fn = codegen->FinalizeFunction(process_batch_streaming_fn);
-  if (process_batch_streaming_fn == NULL) {
-    return Status("PartitionedAggregationNode::CodegenProcessBatchStreaming(): codegen'd "
-        "ProcessBatchStreaming() function failed verification, see log");
-  }
-
-  codegen->AddFunctionToJit(process_batch_streaming_fn,
-      reinterpret_cast<void**>(&process_batch_streaming_fn_));
-  return Status::OK();
-}
-
-// Instantiate required templates.
-template Status PartitionedAggregationNode::AppendSpilledRow<false>(
-    Partition*, TupleRow*);
-template Status PartitionedAggregationNode::AppendSpilledRow<true>(Partition*, TupleRow*);
-}
diff --git a/be/src/exec/streaming-aggregation-node.cc b/be/src/exec/streaming-aggregation-node.cc
new file mode 100644
index 0000000..4ad7820
--- /dev/null
+++ b/be/src/exec/streaming-aggregation-node.cc
@@ -0,0 +1,153 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "exec/streaming-aggregation-node.h"
+
+#include <sstream>
+
+#include "gutil/strings/substitute.h"
+#include "runtime/row-batch.h"
+#include "runtime/runtime-state.h"
+#include "util/runtime-profile-counters.h"
+
+#include "gen-cpp/PlanNodes_types.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+StreamingAggregationNode::StreamingAggregationNode(
+    ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs)
+  : ExecNode(pool, tnode, descs), child_eos_(false) {
+  DCHECK(conjunct_evals_.empty()) << "Preaggs have no conjuncts";
+  DCHECK(!tnode.agg_node.grouping_exprs.empty()) << "Streaming preaggs do grouping";
+  DCHECK(limit_ == -1) << "Preaggs have no limits";
+}
+
+Status StreamingAggregationNode::Init(const TPlanNode& tnode, RuntimeState* state) {
+  RETURN_IF_ERROR(ExecNode::Init(tnode, state));
+  aggregator_.reset(new GroupingAggregator(this, pool_, tnode, state->desc_tbl()));
+  RETURN_IF_ERROR(aggregator_->Init(tnode, state));
+  runtime_profile_->AddChild(aggregator_->runtime_profile());
+  return Status::OK();
+}
+
+Status StreamingAggregationNode::Prepare(RuntimeState* state) {
+  SCOPED_TIMER(runtime_profile_->total_time_counter());
+  RETURN_IF_ERROR(ExecNode::Prepare(state));
+  aggregator_->SetDebugOptions(debug_options_);
+  RETURN_IF_ERROR(aggregator_->Prepare(state));
+  state->CheckAndAddCodegenDisabledMessage(runtime_profile());
+  return Status::OK();
+}
+
+void StreamingAggregationNode::Codegen(RuntimeState* state) {
+  DCHECK(state->ShouldCodegen());
+  ExecNode::Codegen(state);
+  if (IsNodeCodegenDisabled()) return;
+
+  aggregator_->Codegen(state);
+}
+
+Status StreamingAggregationNode::Open(RuntimeState* state) {
+  SCOPED_TIMER(runtime_profile_->total_time_counter());
+  // Open the child before consuming resources in this node.
+  RETURN_IF_ERROR(child(0)->Open(state));
+  RETURN_IF_ERROR(ExecNode::Open(state));
+
+  RETURN_IF_ERROR(aggregator_->Open(state));
+
+  // Streaming preaggregations do all processing in GetNext().
+  return Status::OK();
+}
+
+Status StreamingAggregationNode::GetNext(
+    RuntimeState* state, RowBatch* row_batch, bool* eos) {
+  SCOPED_TIMER(runtime_profile_->total_time_counter());
+  RETURN_IF_ERROR(ExecDebugAction(TExecNodePhase::GETNEXT, state));
+  RETURN_IF_CANCELLED(state);
+  RETURN_IF_ERROR(QueryMaintenance(state));
+
+  if (ReachedLimit()) {
+    *eos = true;
+    return Status::OK();
+  }
+
+  bool aggregator_eos = false;
+  if (!child_eos_) {
+    // For streaming preaggregations, we process rows from the child as we go.
+    RETURN_IF_ERROR(GetRowsStreaming(state, row_batch));
+  } else {
+    RETURN_IF_ERROR(aggregator_->GetNext(state, row_batch, &aggregator_eos));
+  }
+
+  *eos = aggregator_eos && child_eos_;
+  num_rows_returned_ += row_batch->num_rows();
+  COUNTER_SET(rows_returned_counter_, num_rows_returned_);
+  return Status::OK();
+}
+
+Status StreamingAggregationNode::GetRowsStreaming(
+    RuntimeState* state, RowBatch* out_batch) {
+  DCHECK(!child_eos_);
+
+  if (child_batch_ == nullptr) {
+    child_batch_.reset(
+        new RowBatch(child(0)->row_desc(), state->batch_size(), mem_tracker()));
+  }
+
+  do {
+    DCHECK_EQ(out_batch->num_rows(), 0);
+    RETURN_IF_CANCELLED(state);
+    RETURN_IF_ERROR(QueryMaintenance(state));
+
+    RETURN_IF_ERROR(child(0)->GetNext(state, child_batch_.get(), &child_eos_));
+
+    RETURN_IF_ERROR(aggregator_->AddBatchStreaming(state, out_batch, child_batch_.get()));
+    child_batch_->Reset(); // All rows from child_batch_ were processed.
+  } while (out_batch->num_rows() == 0 && !child_eos_);
+
+  if (child_eos_) {
+    child(0)->Close(state);
+    child_batch_.reset();
+    RETURN_IF_ERROR(aggregator_->InputDone());
+  }
+
+  return Status::OK();
+}
+
+Status StreamingAggregationNode::Reset(RuntimeState* state) {
+  DCHECK(false) << "Cannot reset preaggregation";
+  return Status("Cannot reset preaggregation");
+}
+
+void StreamingAggregationNode::Close(RuntimeState* state) {
+  if (is_closed()) return;
+  aggregator_->Close(state);
+  child_batch_.reset();
+  ExecNode::Close(state);
+}
+
+void StreamingAggregationNode::DebugString(
+    int indentation_level, stringstream* out) const {
+  *out << string(indentation_level * 2, ' ');
+  *out << "StreamingAggregationNode("
+       << "aggregator=" << aggregator_->DebugString();
+  ExecNode::DebugString(indentation_level, out);
+  *out << ")";
+}
+} // namespace impala
diff --git a/be/src/exec/streaming-aggregation-node.h b/be/src/exec/streaming-aggregation-node.h
new file mode 100644
index 0000000..8e06b2a
--- /dev/null
+++ b/be/src/exec/streaming-aggregation-node.h
@@ -0,0 +1,85 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef IMPALA_EXEC_STREAMING_AGGREGATION_NODE_H
+#define IMPALA_EXEC_STREAMING_AGGREGATION_NODE_H
+
+#include <memory>
+
+#include "exec/exec-node.h"
+#include "exec/grouping-aggregator.h"
+
+namespace impala {
+
+class RowBatch;
+class RuntimeState;
+
+/// Node for doing streaming partitioned hash aggregation.
+///
+/// This node consumes the input from child(0) during GetNext() and then passes it to the
+/// Aggregator, which does the actual work of aggregating. The aggregator will attempt to
+/// aggregate the rows into its hash table, but if there is not enough memory available or
+/// if the reduction from the aggregation is not very good, it will 'stream' the rows
+/// through and return them without aggregating them instead of spilling. After all of the
+/// input as been processed from child(0), subsequent calls to GetNext() will return any
+/// rows that were aggregated in the Aggregator's hash table.
+///
+/// Since the rows returned by GetNext() may be only partially aggregated if there are
+/// memory contraints, this is a preliminary aggregation step that functions as an
+/// optimization and will always be followed in the plan by an AggregationNode that does
+/// the final aggregation.
+///
+/// This node only supports grouping aggregations.
+class StreamingAggregationNode : public ExecNode {
+ public:
+  StreamingAggregationNode(
+      ObjectPool* pool, const TPlanNode& tnode, const DescriptorTbl& descs);
+
+  virtual Status Init(const TPlanNode& tnode, RuntimeState* state) override;
+  virtual Status Prepare(RuntimeState* state) override;
+  virtual void Codegen(RuntimeState* state) override;
+  virtual Status Open(RuntimeState* state) override;
+  virtual Status GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos) override;
+  virtual Status Reset(RuntimeState* state) override;
+  virtual void Close(RuntimeState* state) override;
+
+  virtual void DebugString(int indentation_level, std::stringstream* out) const override;
+
+ private:
+  /////////////////////////////////////////
+  /// BEGIN: Members that must be Reset()
+
+  /// Row batch used as argument to GetNext() for the child node preaggregations. Store
+  /// in node to avoid reallocating for every GetNext() call when streaming.
+  std::unique_ptr<RowBatch> child_batch_;
+
+  /// True if no more rows to process from child.
+  bool child_eos_;
+
+  std::unique_ptr<GroupingAggregator> aggregator_;
+
+  /// END: Members that must be Reset()
+  /////////////////////////////////////////
+
+  /// Get output rows from child for streaming pre-aggregation. Aggregates some rows with
+  /// hash table and passes through other rows converted into the intermediate
+  /// tuple format. Sets 'child_eos_' once all rows from child have been returned.
+  Status GetRowsStreaming(RuntimeState* state, RowBatch* row_batch) WARN_UNUSED_RESULT;
+};
+} // namespace impala
+
+#endif // IMPALA_EXEC_STREAMING_AGGREGATION_NODE_H


[impala] 01/14: Ignore "IMPALA-7200: Fix missing FILESYSTEM_PREFIX hitting local dataload"

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit c77b648e0f47cf556ee761ae6b83a440f715d772
Author: stiga-huang <hu...@gmail.com>
AuthorDate: Fri Feb 22 04:05:51 2019 -0800

    Ignore "IMPALA-7200: Fix missing FILESYSTEM_PREFIX hitting local dataload"
    
    Change-Id: I5da5ae08eeb8244c8f3ad49e538647200551cb5c
    Reviewed-on: http://gerrit.cloudera.org:8080/12554
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 bin/ignored_commits.json | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/bin/ignored_commits.json b/bin/ignored_commits.json
index a8528aa..bd6ba45 100644
--- a/bin/ignored_commits.json
+++ b/bin/ignored_commits.json
@@ -34,7 +34,9 @@
       { "hash": "56a740c07a6d80921e86fee769033fab5ad1ccf3",
         "comment": "Not applicable to 2.x: 3.x erasure coding"},
       { "hash": "a0257013df735141e5d36f884305af26c92b7c4b",
-        "comment": "Not applicable to 2.x: test not in 2.x"}
+        "comment": "Not applicable to 2.x: test not in 2.x"},
+      { "hash": "c8bfcbd6e826110678df4a699a9e8f05cf57b5d2",
+        "comment": "Not applicable to 2.x: Fix loading test time-zone database for 3.x"}
     ]
   }
 ]


[impala] 14/14: IMPALA-7185: low statestore custom cluster interval

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 93fa95aeada76fe06d88b6c7afbb2f590f18d541
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Mon Jun 18 14:28:13 2018 -0700

    IMPALA-7185: low statestore custom cluster interval
    
    This changes the default statestore interval for the custom cluster
    tests. This can reduce the time taken for the cluster to start and
    metadata to load. On some tests this resulted in saving 5+ seconds
    per test. Overall it shaved around a minute off the custom cluster
    tests.
    
    Testing:
    Ran 10 iterations of the tests.
    
    Change-Id: Ia5d1612283ff420d95b0dd0ca5a2a67f56765f79
    Reviewed-on: http://gerrit.cloudera.org:8080/10845
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 tests/authorization/test_grant_revoke.py  |  7 ++-----
 tests/common/custom_cluster_test_suite.py | 12 ++++++++++--
 2 files changed, 12 insertions(+), 7 deletions(-)

diff --git a/tests/authorization/test_grant_revoke.py b/tests/authorization/test_grant_revoke.py
index 34ee513..36642fb 100644
--- a/tests/authorization/test_grant_revoke.py
+++ b/tests/authorization/test_grant_revoke.py
@@ -100,8 +100,7 @@ class TestGrantRevoke(CustomClusterTestSuite, ImpalaTestSuite):
   @CustomClusterTestSuite.with_args(
       impalad_args="--server_name=server1",
       catalogd_args="--sentry_config=" + SENTRY_CONFIG_FILE +
-                    " --sentry_catalog_polling_frequency_s=1",
-      statestored_args="--statestore_update_frequency_ms=300")
+                    " --sentry_catalog_polling_frequency_s=1")
   def test_role_privilege_case(self, vector):
     """IMPALA-5582: Store sentry privileges in lower case. This
     test grants select privileges to roles assgined to tables/db
@@ -165,9 +164,7 @@ class TestGrantRevoke(CustomClusterTestSuite, ImpalaTestSuite):
   @pytest.mark.execute_serially
   @CustomClusterTestSuite.with_args(
       impalad_args="--server_name=server1",
-      catalogd_args="--sentry_config=" + SENTRY_CONFIG_FILE,
-      statestored_args=("--statestore_heartbeat_frequency_ms=300 "
-                        "--statestore_update_frequency_ms=300"))
+      catalogd_args="--sentry_config=" + SENTRY_CONFIG_FILE)
   def test_role_update(self, vector):
     """IMPALA-5355: The initial update from the statestore has the privileges and roles in
     reverse order if a role was modified, but not the associated privilege. Verify that
diff --git a/tests/common/custom_cluster_test_suite.py b/tests/common/custom_cluster_test_suite.py
index 20037bf..8a0ebb8 100644
--- a/tests/common/custom_cluster_test_suite.py
+++ b/tests/common/custom_cluster_test_suite.py
@@ -39,6 +39,11 @@ CATALOGD_ARGS = 'catalogd_args'
 # Additional args passed to the start-impala-cluster script.
 START_ARGS = 'start_args'
 
+# Run with fast topic updates by default to reduce time to first query running.
+DEFAULT_STATESTORE_ARGS = '--statestore_update_frequency_ms=50 \
+    --statestore_priority_update_frequency_ms=50 \
+    --statestore_heartbeat_frequency_ms=50'
+
 class CustomClusterTestSuite(ImpalaTestSuite):
   """Every test in a test suite deriving from this class gets its own Impala cluster.
   Custom arguments may be passed to the cluster by using the @with_args decorator."""
@@ -88,8 +93,11 @@ class CustomClusterTestSuite(ImpalaTestSuite):
     def decorate(func):
       if impalad_args is not None:
         func.func_dict[IMPALAD_ARGS] = impalad_args
-      if statestored_args is not None:
-        func.func_dict[STATESTORED_ARGS] = statestored_args
+      if statestored_args is None:
+        func.func_dict[STATESTORED_ARGS] = DEFAULT_STATESTORE_ARGS
+      else:
+        func.func_dict[STATESTORED_ARGS] = \
+            DEFAULT_STATESTORE_ARGS + " " + statestored_args
       if catalogd_args is not None:
         func.func_dict[CATALOGD_ARGS] = catalogd_args
       if start_args is not None:


[impala] 02/14: IMPALA-7121: Clean up partitionIds_ from HdfsTable

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 34e444ca86d51b262df97a8c73b351dea6fb78c6
Author: Gabor Kaszab <ga...@cloudera.com>
AuthorDate: Tue Jun 5 18:08:15 2018 +0200

    IMPALA-7121: Clean up partitionIds_ from HdfsTable
    
    The purpose of introducing partitionIds_ member to HdfsTable was to be
    able to return the IDs of all the current partitions in constant time.
    Apparently, partitionMap_ also contains these IDs as the key of the
    map and this is accessible via keySet() also in constant time. It
    seems reasonable then to remove partitionIds_ and use
    partitionMap_.keySet() in getPartitionIds() to save some memory.
    
    Change-Id: I8b5a480e570aeae565fafd4f3e2b279e7a98c7da
    Reviewed-on: http://gerrit.cloudera.org:8080/10654
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 fe/src/main/java/org/apache/impala/catalog/HdfsTable.java    | 12 ++++--------
 .../java/org/apache/impala/planner/HdfsPartitionPruner.java  |  6 ++----
 2 files changed, 6 insertions(+), 12 deletions(-)

diff --git a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
index 13521df..34896cb 100644
--- a/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
+++ b/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java
@@ -185,9 +185,6 @@ public class HdfsTable extends Table implements FeFsTable {
   // table metadata loading.
   private final HashMap<String, HdfsPartition> nameToPartitionMap_ = Maps.newHashMap();
 
-  // Store all the partition ids of an HdfsTable.
-  private final HashSet<Long> partitionIds_ = Sets.newHashSet();
-
   // The partition used as a prototype when creating new partitions during
   // insertion. New partitions inherit file format and other settings from
   // the prototype.
@@ -586,8 +583,11 @@ public class HdfsTable extends Table implements FeFsTable {
     return partitionLocationCompressor_;
   }
 
+  // Returns an unmodifiable set of the partition IDs from partitionMap_.
   @Override // FeFsTable
-  public Set<Long> getPartitionIds() { return partitionIds_; }
+  public Set<Long> getPartitionIds() {
+    return Collections.unmodifiableSet(partitionMap_.keySet());
+  }
 
   @Override // FeFsTable
   public TreeMap<LiteralExpr, HashSet<Long>> getPartitionValueMap(int i) {
@@ -745,7 +745,6 @@ public class HdfsTable extends Table implements FeFsTable {
    * Clear the partitions of an HdfsTable and the associated metadata.
    */
   private void resetPartitions() {
-    partitionIds_.clear();
     partitionMap_.clear();
     nameToPartitionMap_.clear();
     partitionValuesMap_.clear();
@@ -1093,7 +1092,6 @@ public class HdfsTable extends Table implements FeFsTable {
    */
   private void updatePartitionMdAndColStats(HdfsPartition partition) {
     if (partition.getPartitionValues().size() != numClusteringCols_) return;
-    partitionIds_.add(partition.getId());
     nameToPartitionMap_.put(partition.getPartitionName(), partition);
     if (!isStoredInImpaladCatalogCache()) return;
     for (int i = 0; i < partition.getPartitionValues().size(); ++i) {
@@ -1141,8 +1139,6 @@ public class HdfsTable extends Table implements FeFsTable {
     Preconditions.checkArgument(partition.getPartitionValues().size() ==
         numClusteringCols_);
     Long partitionId = partition.getId();
-    // Remove the partition id from the list of partition ids and other mappings.
-    partitionIds_.remove(partitionId);
     partitionMap_.remove(partitionId);
     nameToPartitionMap_.remove(partition.getPartitionName());
     if (!isStoredInImpaladCatalogCache()) return partition;
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java b/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java
index 07a2636..1584d44 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsPartitionPruner.java
@@ -276,17 +276,15 @@ public class HdfsPartitionPruner {
     }
     if (op == Operator.DISTINCT_FROM) {
       // Case: SlotRef IS DISTINCT FROM Literal
+      matchingIds.addAll(tbl_.getPartitionIds());
       if (literal instanceof NullLiteral) {
-        matchingIds.addAll(tbl_.getPartitionIds());
         Set<Long> nullIds = tbl_.getNullPartitionIds(partitionPos);
         matchingIds.removeAll(nullIds);
-        return matchingIds;
       } else {
-        matchingIds.addAll(tbl_.getPartitionIds());
         HashSet<Long> ids = partitionValueMap.get(literal);
         if (ids != null) matchingIds.removeAll(ids);
-        return matchingIds;
       }
+      return matchingIds;
     }
     if (op == Operator.NE) {
       // Case: SlotRef != Literal


[impala] 07/14: IMPALA-7210: global debug actions should be case insensitive

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 0091cd6027b1e64d0209f9fae08271ed47cc524a
Author: Dan Hecht <dh...@cloudera.com>
AuthorDate: Mon Jun 25 14:46:48 2018 -0700

    IMPALA-7210: global debug actions should be case insensitive
    
    The ExecNode debug actions don't care about case so better
    to be consistent.
    
    Testing: verify that this works:
      set debug_action=coord_before_exec_rpc:sleep@1000
    
    Change-Id: Ia3f738caeb602afce4ca638ce354302e521187dc
    Reviewed-on: http://gerrit.cloudera.org:8080/10814
    Reviewed-by: Dan Hecht <dh...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/util/debug-util.cc | 11 +++++++----
 1 file changed, 7 insertions(+), 4 deletions(-)

diff --git a/be/src/util/debug-util.cc b/be/src/util/debug-util.cc
index f305943..052d70d 100644
--- a/be/src/util/debug-util.cc
+++ b/be/src/util/debug-util.cc
@@ -20,6 +20,7 @@
 #include <iomanip>
 #include <random>
 #include <sstream>
+#include <boost/algorithm/string.hpp>
 #include <boost/tokenizer.hpp>
 
 #include "common/version.h"
@@ -42,6 +43,7 @@ extern void DumpStackTraceToString(std::string* s);
 
 #include "common/names.h"
 
+using boost::algorithm::iequals;
 using boost::char_separator;
 using boost::is_any_of;
 using boost::split;
@@ -330,20 +332,21 @@ Status DebugActionImpl(
       query_options.debug_action);
   static const char ERROR_MSG[] = "Invalid debug_action $0:$1 ($2)";
   for (const vector<string>& components : action_list) {
-    if (components.size() != 2 || components[0].compare(label) != 0) continue;
+    // size() != 2 check filters out ExecNode debug actions.
+    if (components.size() != 2 || !iequals(components[0], label)) continue;
     // 'tokens' becomes {command, param0, param1, ... }
     vector<string> tokens = TokenizeDebugActionParams(components[1]);
     DCHECK_GE(tokens.size(), 1);
     const string& cmd = tokens[0];
     int sleep_millis = 0;
-    if (cmd.compare("SLEEP") == 0) {
+    if (iequals(cmd, "SLEEP")) {
       // SLEEP@<millis>
       if (tokens.size() != 2) {
         return Status(Substitute(ERROR_MSG, components[0], components[1],
                 "expected SLEEP@<ms>"));
       }
       sleep_millis = atoi(tokens[1].c_str());
-    } else if (cmd.compare("JITTER") == 0) {
+    } else if (iequals(cmd, "JITTER")) {
       // JITTER@<millis>[@<probability>}
       if (tokens.size() < 2 || tokens.size() > 3) {
         return Status(Substitute(ERROR_MSG, components[0], components[1],
@@ -359,7 +362,7 @@ Status DebugActionImpl(
         if (!should_execute) continue;
       }
       sleep_millis = rand() % (max_millis + 1);
-    } else if (cmd.compare("FAIL") == 0) {
+    } else if (iequals(cmd, "FAIL")) {
       // FAIL[@<probability>]
       if (tokens.size() > 2) {
         return Status(Substitute(ERROR_MSG, components[0], components[1],


[impala] 11/14: IMPALA-7215: Implement a templatized CountingBarrier

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit f823c46099aef28fe01c7348f8bd94c11b7fb7d3
Author: Sailesh Mukil <sa...@apache.org>
AuthorDate: Tue Jun 26 11:20:12 2018 -0700

    IMPALA-7215: Implement a templatized CountingBarrier
    
    Currently, our CountingBarrier util only notifies a 'bool' value
    and uses an underlying Promise<bool>.
    
    We're seeing cases in code where we might want to be notified of a
    different kind of Promise (other than bool). We add a templatized
    class TypedCountingBarrier<T> and convert CountingBarrier to use the
    TypedCountingBarrier<T> internally.
    
    This was identified while working on IMPALA-7163.
    
    Testing: Ran 'core' tests.
    
    Change-Id: I05fc79228250408ae16481ae7ff3491a90d26b8e
    Reviewed-on: http://gerrit.cloudera.org:8080/10827
    Reviewed-by: Sailesh Mukil <sa...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exec/hdfs-scan-node.cc  |  2 +-
 be/src/rpc/rpc-mgr-test.cc     |  6 ++--
 be/src/runtime/coordinator.cc  |  2 +-
 be/src/util/counting-barrier.h | 75 +++++++++++++++++++++++++++++++-----------
 be/src/util/hdfs-bulk-ops.cc   |  2 +-
 5 files changed, 62 insertions(+), 25 deletions(-)

diff --git a/be/src/exec/hdfs-scan-node.cc b/be/src/exec/hdfs-scan-node.cc
index fe2eee7..e9c0864 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -81,7 +81,7 @@ Status HdfsScanNode::GetNext(RuntimeState* state, RowBatch* row_batch, bool* eos
     RETURN_IF_ERROR(IssueInitialScanRanges(state));
 
     // Release the scanner threads
-    ranges_issued_barrier_.Notify();
+    discard_result(ranges_issued_barrier_.Notify());
 
     if (progress_.done()) SetDone();
   }
diff --git a/be/src/rpc/rpc-mgr-test.cc b/be/src/rpc/rpc-mgr-test.cc
index e90838e..880cb45 100644
--- a/be/src/rpc/rpc-mgr-test.cc
+++ b/be/src/rpc/rpc-mgr-test.cc
@@ -225,8 +225,10 @@ TEST_F(RpcMgrTest, AsyncCall) {
     ScanMemResponsePB response;
     SetupScanMemRequest(&request, &controller);
     CountingBarrier barrier(1);
-    scan_mem_proxy->ScanMemAsync(request, &response, &controller,
-        [barrier_ptr = &barrier]() { barrier_ptr->Notify(); });
+    scan_mem_proxy->ScanMemAsync(
+        request, &response, &controller, [barrier_ptr = &barrier]() {
+          discard_result(barrier_ptr->Notify());
+        });
     // TODO: Inject random cancellation here.
     barrier.Wait();
     ASSERT_TRUE(controller.status().ok()) << controller.status().ToString();
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index a240e85..020b950 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -706,7 +706,7 @@ Status Coordinator::UpdateBackendExecStatus(const TReportExecStatusParams& param
               TNetworkAddressToString(backend_state->impalad_address())));
     }
     // We've applied all changes from the final status report - notify waiting threads.
-    backend_exec_complete_barrier_->Notify();
+    discard_result(backend_exec_complete_barrier_->Notify());
   }
   // If query execution has terminated, return a cancelled status to force the fragment
   // instance to stop executing.
diff --git a/be/src/util/counting-barrier.h b/be/src/util/counting-barrier.h
index 827c526..c1bc805 100644
--- a/be/src/util/counting-barrier.h
+++ b/be/src/util/counting-barrier.h
@@ -23,51 +23,88 @@
 
 namespace impala {
 
-/// Allows clients to wait for the arrival of a fixed number of notifications before they
-/// are allowed to continue.
-class CountingBarrier {
+/// Allows clients to wait for the arrival of a fixed number of notifications after which
+/// they are returned a value of type 'T' and allowed to continue.
+template <typename T>
+class TypedCountingBarrier {
  public:
-  /// Initialises the CountingBarrier with `count` pending notifications.
-  CountingBarrier(int32_t count) : count_(count) {
-    DCHECK_GT(count, 0);
-  }
+  /// Initialises the TypedCountingBarrier with `count` pending notifications.
+  TypedCountingBarrier(int32_t count) : count_(count) { DCHECK_GT(count, 0); }
 
   /// Sends one notification, decrementing the number of pending notifications by one.
   /// Returns the remaining pending notifications.
-  int32_t Notify() {
+  /// If this is the final notifier, it unblocks Wait() with the returned value as
+  /// 'promise_value'.
+  int32_t Notify(const T& promise_value) {
     int32_t result = count_.Add(-1);
-    if (result == 0) promise_.Set(true);
+    if (result == 0) promise_.Set(promise_value);
     return result;
   }
 
-  /// Sets the number of pending notifications to 0 and unblocks Wait().
-  void NotifyRemaining() {
+  /// Sets the number of pending notifications to 0 and unblocks Wait() with the returned
+  /// value as 'promise_value'.
+  void NotifyRemaining(const T& promise_value) {
     while (true) {
       int32_t value = count_.Load();
       if (value <= 0) return;  // count_ can legitimately drop below 0
       if (count_.CompareAndSwap(value, 0)) {
-        promise_.Set(true);
+        promise_.Set(promise_value);
         return;
       }
     }
   }
 
-  /// Blocks until all notifications are received.
-  void Wait() { promise_.Get(); }
+  /// Blocks until all notifications are received. Returns the value set by
+  /// Notify() or NotifyRemaining().
+  const T& Wait() { return promise_.Get(); }
 
   /// Blocks until all notifications are received, or until 'timeout_ms' passes, in which
-  /// case '*timed_out' will be true.
-  void Wait(int64_t timeout_ms, bool* timed_out) { promise_.Get(timeout_ms, timed_out); }
+  /// case '*timed_out' will be true. If '*timed_out' is false, then returns the value set
+  /// by Notify() or NotifyRemaining().
+  const T& Wait(int64_t timeout_ms, bool* timed_out) {
+    return promise_.Get(timeout_ms, timed_out);
+  }
 
   int32_t pending() const { return count_.Load(); }
 
  private:
   /// Used to signal waiters when all notifications are received.
-  Promise<bool> promise_;
+  Promise<T> promise_;
 
   /// The number of pending notifications remaining.
   AtomicInt32 count_;
 
+  DISALLOW_COPY_AND_ASSIGN(TypedCountingBarrier);
+};
+
+/// Wrapper around TypedCountingBarrier<T> which allows clients to wait for the arrival
+/// of a fixed number of notifications after which they are allowed to continue.
+class CountingBarrier {
+ public:
+  /// Initialises the CountingBarrier with `count` pending notifications.
+  CountingBarrier(int32_t count) : barrier_(count) { DCHECK_GT(count, 0); }
+
+  /// Sends one notification, decrementing the number of pending notifications by one.
+  /// Returns the remaining pending notifications.
+  int32_t Notify() { return barrier_.Notify(true); }
+
+  /// Sets the number of pending notifications to 0 and unblocks Wait().
+  void NotifyRemaining() { barrier_.NotifyRemaining(true); }
+
+  /// Blocks until all notifications are received.
+  void Wait() { discard_result(barrier_.Wait()); }
+
+  /// Blocks until all notifications are received, or until 'timeout_ms' passes, in which
+  /// case '*timed_out' will be true.
+  void Wait(int64_t timeout_ms, bool* timed_out) {
+    discard_result(barrier_.Wait(timeout_ms, timed_out));
+  }
+
+  int32_t pending() const { return barrier_.pending(); }
+
+ private:
+  TypedCountingBarrier<bool> barrier_;
+
   DISALLOW_COPY_AND_ASSIGN(CountingBarrier);
 };
 
@@ -79,9 +116,7 @@ class NotifyBarrierOnExit {
     DCHECK(b != NULL);
   }
 
-  ~NotifyBarrierOnExit() {
-    barrier->Notify();
-  }
+  ~NotifyBarrierOnExit() { discard_result(barrier->Notify()); }
 
  private:
   CountingBarrier* barrier;
diff --git a/be/src/util/hdfs-bulk-ops.cc b/be/src/util/hdfs-bulk-ops.cc
index c5c7ad5..53d5c7a 100644
--- a/be/src/util/hdfs-bulk-ops.cc
+++ b/be/src/util/hdfs-bulk-ops.cc
@@ -182,7 +182,7 @@ void HdfsOperationSet::AddError(const string& err, const HdfsOp* op) {
 }
 
 void HdfsOperationSet::MarkOneOpDone() {
-  ops_complete_barrier_->Notify();
+  discard_result(ops_complete_barrier_->Notify());
 }
 
 bool HdfsOperationSet::ShouldAbort() {


[impala] 04/14: IMPALA-7207: make Coordinator::exec_state_ an atomic enum

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit ac4219563b5c0a454946f41b3f0c0e54d4e454bf
Author: Dan Hecht <dh...@cloudera.com>
AuthorDate: Mon Jun 25 10:47:54 2018 -0700

    IMPALA-7207: make Coordinator::exec_state_ an atomic enum
    
    That allows us to avoid taking the lock in cases where only
    the exec_state_ field needs to be read (as opposed to needing
    to read both exec_state_ and exec_status_). In particular,
    it avoids the lock on the non-terminating paths, which is
    the common case.
    
    Change-Id: Ie6c5d5c6ccfdfd533cd0607aab6f554e664b90ac
    Reviewed-on: http://gerrit.cloudera.org:8080/10811
    Reviewed-by: Tim Armstrong <ta...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/common/atomic.h                   |  3 ++-
 be/src/runtime/coordinator.cc            | 21 +++++++--------------
 be/src/runtime/coordinator.h             | 12 ++++++++----
 be/src/runtime/fragment-instance-state.h |  3 ++-
 4 files changed, 19 insertions(+), 20 deletions(-)

diff --git a/be/src/common/atomic.h b/be/src/common/atomic.h
index 4c72826..5aa5f86 100644
--- a/be/src/common/atomic.h
+++ b/be/src/common/atomic.h
@@ -153,11 +153,12 @@ class AtomicEnum {
       "Underlying enum type must fit into 4 bytes");
 
  public:
+  AtomicEnum(T initial) : enum_(static_cast<int32_t>(initial)) {}
   /// Atomic load with "acquire" memory-ordering semantic.
   ALWAYS_INLINE T Load() const { return static_cast<T>(enum_.Load()); }
 
   /// Atomic store with "release" memory-ordering semantic.
-  ALWAYS_INLINE void Store(T val) { enum_.Store(val); }
+  ALWAYS_INLINE void Store(T val) { enum_.Store(static_cast<int32_t>(val)); }
 
  private:
   internal::AtomicInt<int32_t> enum_;
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index 22930a4..3489312 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -76,7 +76,7 @@ Coordinator::Coordinator(
 
 Coordinator::~Coordinator() {
   // Must have entered a terminal exec state guaranteeing resources were released.
-  DCHECK_NE(exec_state_, ExecState::EXECUTING);
+  DCHECK_NE(exec_state_.Load(), ExecState::EXECUTING);
   DCHECK_LE(backend_exec_complete_barrier_->pending(), 0);
   // Release the coordinator's reference to the query control structures.
   if (query_state_ != nullptr) {
@@ -450,9 +450,9 @@ Status Coordinator::SetNonErrorTerminalState(const ExecState state) {
   {
     lock_guard<SpinLock> l(exec_state_lock_);
     // May have already entered a terminal state, in which case nothing to do.
-    if (exec_state_ != ExecState::EXECUTING) return exec_status_;
+    if (exec_state_.Load() != ExecState::EXECUTING) return exec_status_;
     DCHECK(exec_status_.ok()) << exec_status_;
-    exec_state_ = state;
+    exec_state_.Store(state);
     if (state == ExecState::CANCELLED) exec_status_ = Status::CANCELLED;
     ret_status = exec_status_;
   }
@@ -468,13 +468,13 @@ Status Coordinator::UpdateExecState(const Status& status,
   ExecState old_state, new_state;
   {
     lock_guard<SpinLock> l(exec_state_lock_);
-    old_state = exec_state_;
+    old_state = exec_state_.Load();
     if (old_state == ExecState::EXECUTING) {
       DCHECK(exec_status_.ok()) << exec_status_;
       if (!status.ok()) {
         // Error while executing - go to ERROR state.
         exec_status_ = status;
-        exec_state_ = ExecState::ERROR;
+        exec_state_.Store(ExecState::ERROR);
       }
     } else if (old_state == ExecState::RETURNED_RESULTS) {
       // Already returned all results. Leave exec status as ok, stay in this state.
@@ -492,7 +492,7 @@ Status Coordinator::UpdateExecState(const Status& status,
         exec_status_ = status;
       }
     }
-    new_state = exec_state_;
+    new_state = exec_state_.Load();
     ret_status = exec_status_;
   }
   // Log interesting status: a non-cancelled error or a cancellation if was executing.
@@ -508,11 +508,6 @@ Status Coordinator::UpdateExecState(const Status& status,
   return ret_status;
 }
 
-bool Coordinator::ReturnedAllResults() {
-  lock_guard<SpinLock> l(exec_state_lock_);
-  return exec_state_ == ExecState::RETURNED_RESULTS;
-}
-
 void Coordinator::HandleExecStateTransition(
     const ExecState old_state, const ExecState new_state) {
   static const unordered_map<ExecState, const char *> exec_state_to_event{
@@ -623,9 +618,7 @@ Status Coordinator::GetNext(QueryResultSet* results, int max_rows, bool* eos) {
   DCHECK(has_called_wait_);
   SCOPED_TIMER(query_profile_->total_time_counter());
 
-  // exec_state_lock_ not needed here though since this path won't execute concurrently
-  // with itself or DML finalization.
-  if (exec_state_ == ExecState::RETURNED_RESULTS) {
+  if (ReturnedAllResults()) {
     // Nothing left to do: already in a terminal state and no more results.
     *eos = true;
     return Status::OK();
diff --git a/be/src/runtime/coordinator.h b/be/src/runtime/coordinator.h
index 5bb399f..a0dce35 100644
--- a/be/src/runtime/coordinator.h
+++ b/be/src/runtime/coordinator.h
@@ -29,7 +29,6 @@
 #include "gen-cpp/Frontend_types.h"
 #include "gen-cpp/Types_types.h"
 #include "runtime/dml-exec-state.h"
-#include "util/condition-variable.h"
 #include "util/progress-updater.h"
 #include "util/runtime-profile-counters.h"
 #include "util/spinlock.h"
@@ -276,7 +275,10 @@ class Coordinator { // NOLINT: The member variables could be re-ordered to save
   /// cancellation. Initialized in StartBackendExec().
   boost::scoped_ptr<CountingBarrier> backend_exec_complete_barrier_;
 
-  SpinLock exec_state_lock_; // protects exec-state_ and exec_status_
+  // Protects exec_state_ and exec_status_. exec_state_ can be read independently via
+  // the atomic, but the lock is held when writing either field and when reading both
+  // fields together.
+  SpinLock exec_state_lock_;
 
   /// EXECUTING: in-flight; the only non-terminal state
   /// RETURNED_RESULTS: GetNext() set eos to true, or for DML, the request is complete
@@ -285,7 +287,7 @@ class Coordinator { // NOLINT: The member variables could be re-ordered to save
   enum class ExecState {
     EXECUTING, RETURNED_RESULTS, CANCELLED, ERROR
   };
-  ExecState exec_state_ = ExecState::EXECUTING;
+  AtomicEnum<ExecState> exec_state_{ExecState::EXECUTING};
 
   /// Overall execution status; only set on exec_state_ transitions:
   /// - EXECUTING: OK
@@ -357,7 +359,9 @@ class Coordinator { // NOLINT: The member variables could be re-ordered to save
 
   /// Return true if 'exec_state_' is RETURNED_RESULTS.
   /// TODO: remove with IMPALA-6984.
-  bool ReturnedAllResults() WARN_UNUSED_RESULT;
+  bool ReturnedAllResults() WARN_UNUSED_RESULT {
+    return exec_state_.Load() == ExecState::RETURNED_RESULTS;
+  }
 
   /// Return the string representation of 'state'.
   static const char* ExecStateToString(const ExecState state);
diff --git a/be/src/runtime/fragment-instance-state.h b/be/src/runtime/fragment-instance-state.h
index 8295c8f..df27f9c 100644
--- a/be/src/runtime/fragment-instance-state.h
+++ b/be/src/runtime/fragment-instance-state.h
@@ -204,7 +204,8 @@ class FragmentInstanceState {
 
   /// The current state of this fragment instance's execution. Only updated by the
   /// fragment instance thread in UpdateState() and read by the profile reporting threads.
-  AtomicEnum<TFInstanceExecState::type> current_state_;
+  AtomicEnum<TFInstanceExecState::type> current_state_{
+    TFInstanceExecState::WAITING_FOR_EXEC};
 
   /// Output sink for rows sent to this fragment. Created in Prepare(), lives in
   /// obj_pool().


[impala] 08/14: IMPALA-6816: minimise calls to GetMinSubscriberTopicVersion()

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit a29bec1902ffa1ccfb0539a40b874f7650f6b56e
Author: Tim Armstrong <ta...@cloudera.com>
AuthorDate: Mon Jun 11 17:22:35 2018 -0700

    IMPALA-6816: minimise calls to GetMinSubscriberTopicVersion()
    
    min_subscriber_topic_version is expensive to compute (requires iterating
    over all subscribers to compute) but is only used by one
    subscriber/topic pair: Impalads receiving catalog topic updates.
    
    This patch implements a simple fix - only compute it if a subscriber
    asks for it. A more complex alternative would be to maintain
    a priority queue of subscriber versions, but that didn't seem worth
    the the complexity and risk of bugs.
    
    Testing:
    Add a statestore test to validate the versions. It looks like we had a
    pre-existing test gap for validating min_subscriber_topic_version so
    the test is mainly focused on adding that coverage.
    
    Ran core tests with DEBUG and ASAN.
    
    Change-Id: I8ee7cb2355ba1049b9081e0df344ac41aa4ebeb1
    Reviewed-on: http://gerrit.cloudera.org:8080/10705
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/catalog/catalog-server.cc           |  2 +-
 be/src/scheduling/admission-controller.cc  |  9 +--
 be/src/scheduling/scheduler.cc             |  2 +-
 be/src/service/impala-server.cc            | 11 ++--
 be/src/statestore/statestore-subscriber.cc |  7 ++-
 be/src/statestore/statestore-subscriber.h  |  6 +-
 be/src/statestore/statestore.cc            | 50 +++++++++-------
 be/src/statestore/statestore.h             |  8 ++-
 common/thrift/StatestoreService.thrift     |  6 ++
 tests/statestore/test_statestore.py        | 93 ++++++++++++++++++++++++++++++
 10 files changed, 160 insertions(+), 34 deletions(-)

diff --git a/be/src/catalog/catalog-server.cc b/be/src/catalog/catalog-server.cc
index e74db75..a17478f 100644
--- a/be/src/catalog/catalog-server.cc
+++ b/be/src/catalog/catalog-server.cc
@@ -186,7 +186,7 @@ Status CatalogServer::Start() {
 
   StatestoreSubscriber::UpdateCallback cb =
       bind<void>(mem_fn(&CatalogServer::UpdateCatalogTopicCallback), this, _1, _2);
-  status = statestore_subscriber_->AddTopic(IMPALA_CATALOG_TOPIC, false, cb);
+  status = statestore_subscriber_->AddTopic(IMPALA_CATALOG_TOPIC, false, false, cb);
   if (!status.ok()) {
     status.AddDetail("CatalogService failed to start");
     return status;
diff --git a/be/src/scheduling/admission-controller.cc b/be/src/scheduling/admission-controller.cc
index ac176f4..a53d1a0 100644
--- a/be/src/scheduling/admission-controller.cc
+++ b/be/src/scheduling/admission-controller.cc
@@ -18,7 +18,6 @@
 #include "scheduling/admission-controller.h"
 
 #include <boost/algorithm/string.hpp>
-#include <boost/bind.hpp>
 #include <boost/mem_fn.hpp>
 #include <gutil/strings/substitute.h>
 
@@ -243,9 +242,11 @@ AdmissionController::~AdmissionController() {
 Status AdmissionController::Init() {
   RETURN_IF_ERROR(Thread::Create("scheduling", "admission-thread",
       &AdmissionController::DequeueLoop, this, &dequeue_thread_));
-  StatestoreSubscriber::UpdateCallback cb =
-    bind<void>(mem_fn(&AdmissionController::UpdatePoolStats), this, _1, _2);
-  Status status = subscriber_->AddTopic(Statestore::IMPALA_REQUEST_QUEUE_TOPIC, true, cb);
+  auto cb = [this](
+      const StatestoreSubscriber::TopicDeltaMap& state,
+      vector<TTopicDelta>* topic_updates) { UpdatePoolStats(state, topic_updates); };
+  Status status =
+      subscriber_->AddTopic(Statestore::IMPALA_REQUEST_QUEUE_TOPIC, true, false, cb);
   if (!status.ok()) {
     status.AddDetail("AdmissionController failed to register request queue topic");
   }
diff --git a/be/src/scheduling/scheduler.cc b/be/src/scheduling/scheduler.cc
index 5a67a74..950d218 100644
--- a/be/src/scheduling/scheduler.cc
+++ b/be/src/scheduling/scheduler.cc
@@ -88,7 +88,7 @@ Status Scheduler::Init(const TNetworkAddress& backend_address,
     StatestoreSubscriber::UpdateCallback cb =
         bind<void>(mem_fn(&Scheduler::UpdateMembership), this, _1, _2);
     Status status = statestore_subscriber_->AddTopic(
-        Statestore::IMPALA_MEMBERSHIP_TOPIC, true, cb);
+        Statestore::IMPALA_MEMBERSHIP_TOPIC, true, false, cb);
     if (!status.ok()) {
       status.AddDetail("Scheduler failed to register membership topic");
       return status;
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index ce6c2c3..c2b8300 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -358,12 +358,12 @@ ImpalaServer::ImpalaServer(ExecEnv* exec_env)
 
   // Register the membership callback if running in a real cluster.
   if (!TestInfo::is_test()) {
-    auto cb = [this] (const StatestoreSubscriber::TopicDeltaMap& state,
-         vector<TTopicDelta>* topic_updates) {
+    auto cb = [this](const StatestoreSubscriber::TopicDeltaMap& state,
+        vector<TTopicDelta>* topic_updates) {
       this->MembershipCallback(state, topic_updates);
     };
-    ABORT_IF_ERROR(
-        exec_env->subscriber()->AddTopic(Statestore::IMPALA_MEMBERSHIP_TOPIC, true, cb));
+    ABORT_IF_ERROR(exec_env->subscriber()->AddTopic(
+        Statestore::IMPALA_MEMBERSHIP_TOPIC, true, false, cb));
 
     if (FLAGS_is_coordinator && !FLAGS_use_local_catalog) {
       auto catalog_cb = [this] (const StatestoreSubscriber::TopicDeltaMap& state,
@@ -371,7 +371,7 @@ ImpalaServer::ImpalaServer(ExecEnv* exec_env)
         this->CatalogUpdateCallback(state, topic_updates);
       };
       ABORT_IF_ERROR(exec_env->subscriber()->AddTopic(
-            CatalogServer::IMPALA_CATALOG_TOPIC, true, catalog_cb));
+          CatalogServer::IMPALA_CATALOG_TOPIC, true, true, catalog_cb));
     }
   }
 
@@ -1483,6 +1483,7 @@ void ImpalaServer::CatalogUpdateCallback(
   // Always update the minimum subscriber version for the catalog topic.
   {
     unique_lock<mutex> unique_lock(catalog_version_lock_);
+    DCHECK(delta.__isset.min_subscriber_topic_version);
     min_subscriber_catalog_topic_version_ = delta.min_subscriber_topic_version;
   }
   catalog_version_update_cv_.NotifyAll();
diff --git a/be/src/statestore/statestore-subscriber.cc b/be/src/statestore/statestore-subscriber.cc
index a27d1ae..4351589 100644
--- a/be/src/statestore/statestore-subscriber.cc
+++ b/be/src/statestore/statestore-subscriber.cc
@@ -137,7 +137,8 @@ StatestoreSubscriber::StatestoreSubscriber(const std::string& subscriber_id,
 }
 
 Status StatestoreSubscriber::AddTopic(const Statestore::TopicId& topic_id,
-    bool is_transient, const UpdateCallback& callback) {
+    bool is_transient, bool populate_min_subscriber_topic_version,
+    const UpdateCallback& callback) {
   lock_guard<shared_mutex> exclusive_lock(lock_);
   if (is_registered_) return Status("Subscriber already started, can't add new topic");
   TopicRegistration& registration = topic_registrations_[topic_id];
@@ -150,6 +151,8 @@ Status StatestoreSubscriber::AddTopic(const Statestore::TopicId& topic_id,
     registration.update_interval_timer.Start();
   }
   registration.is_transient = is_transient;
+  registration.populate_min_subscriber_topic_version =
+      populate_min_subscriber_topic_version;
   return Status::OK();
 }
 
@@ -163,6 +166,8 @@ Status StatestoreSubscriber::Register() {
     TTopicRegistration thrift_topic;
     thrift_topic.topic_name = registration.first;
     thrift_topic.is_transient = registration.second.is_transient;
+    thrift_topic.populate_min_subscriber_topic_version =
+        registration.second.populate_min_subscriber_topic_version;
     request.topic_registrations.push_back(thrift_topic);
   }
 
diff --git a/be/src/statestore/statestore-subscriber.h b/be/src/statestore/statestore-subscriber.h
index f102cae..7621592 100644
--- a/be/src/statestore/statestore-subscriber.h
+++ b/be/src/statestore/statestore-subscriber.h
@@ -112,7 +112,7 @@ class StatestoreSubscriber {
   /// Must be called before Start(), in which case it will return
   /// Status::OK. Otherwise an error will be returned.
   Status AddTopic(const Statestore::TopicId& topic_id, bool is_transient,
-      const UpdateCallback& callback);
+      bool populate_min_subscriber_topic_version, const UpdateCallback& callback);
 
   /// Registers this subscriber with the statestore, and starts the
   /// heartbeat service, as well as a thread to check for failure and
@@ -212,6 +212,10 @@ class StatestoreSubscriber {
     /// it makes will be deleted upon failure or disconnection.
     bool is_transient = false;
 
+    /// Whether this subscriber needs the min_subscriber_topic_version field to be filled
+    /// in on updates.
+    bool populate_min_subscriber_topic_version = false;
+
     /// The last version of the topic this subscriber processed.
     /// -1 if no updates have been processed yet.
     int64_t current_topic_version = -1;
diff --git a/be/src/statestore/statestore.cc b/be/src/statestore/statestore.cc
index 90ea167..a58aec1 100644
--- a/be/src/statestore/statestore.cc
+++ b/be/src/statestore/statestore.cc
@@ -300,12 +300,14 @@ void Statestore::Topic::ToJson(Document* document, Value* topic_json) {
 Statestore::Subscriber::Subscriber(const SubscriberId& subscriber_id,
     const RegistrationId& registration_id, const TNetworkAddress& network_address,
     const vector<TTopicRegistration>& subscribed_topics)
-    : subscriber_id_(subscriber_id),
-      registration_id_(registration_id),
-      network_address_(network_address) {
-  for (const TTopicRegistration& topic: subscribed_topics) {
-    GetTopicsMapForId(topic.topic_name)->emplace(piecewise_construct,
-        forward_as_tuple(topic.topic_name), forward_as_tuple(topic.is_transient));
+  : subscriber_id_(subscriber_id),
+    registration_id_(registration_id),
+    network_address_(network_address) {
+  for (const TTopicRegistration& topic : subscribed_topics) {
+    GetTopicsMapForId(topic.topic_name)
+        ->emplace(piecewise_construct, forward_as_tuple(topic.topic_name),
+            forward_as_tuple(
+                topic.is_transient, topic.populate_min_subscriber_topic_version));
   }
 }
 
@@ -697,18 +699,22 @@ Status Statestore::SendTopicUpdate(Subscriber* subscriber, UpdateKind update_kin
   return Status::OK();
 }
 
-void Statestore::GatherTopicUpdates(const Subscriber& subscriber,
-    UpdateKind update_kind, TUpdateStateRequest* update_state_request) {
+void Statestore::GatherTopicUpdates(const Subscriber& subscriber, UpdateKind update_kind,
+    TUpdateStateRequest* update_state_request) {
+  DCHECK(update_kind == UpdateKind::TOPIC_UPDATE
+      || update_kind == UpdateKind::PRIORITY_TOPIC_UPDATE)
+      << static_cast<int>(update_kind);
+  // Indices into update_state_request->topic_deltas where we need to populate
+  // 'min_subscriber_topic_version'. GetMinSubscriberTopicVersion() is somewhat
+  // expensive so we want to avoid calling it unless necessary.
+  vector<TTopicDelta*> deltas_needing_min_version;
   {
-    DCHECK(update_kind == UpdateKind::TOPIC_UPDATE
-        || update_kind ==  UpdateKind::PRIORITY_TOPIC_UPDATE)
-        << static_cast<int>(update_kind);
     const bool is_priority = update_kind == UpdateKind::PRIORITY_TOPIC_UPDATE;
-    const Subscriber::Topics& subscribed_topics = is_priority
-        ? subscriber.priority_subscribed_topics()
-        : subscriber.non_priority_subscribed_topics();
+    const Subscriber::Topics& subscribed_topics = is_priority ?
+        subscriber.priority_subscribed_topics() :
+        subscriber.non_priority_subscribed_topics();
     shared_lock<shared_mutex> l(topics_map_lock_);
-    for (const auto& subscribed_topic: subscribed_topics) {
+    for (const auto& subscribed_topic : subscribed_topics) {
       auto topic_it = topics_.find(subscribed_topic.first);
       DCHECK(topic_it != topics_.end());
       TopicEntry::Version last_processed_version =
@@ -718,16 +724,20 @@ void Statestore::GatherTopicUpdates(const Subscriber& subscriber,
           update_state_request->topic_deltas[subscribed_topic.first];
       topic_delta.topic_name = subscribed_topic.first;
       topic_it->second.BuildDelta(subscriber.id(), last_processed_version, &topic_delta);
+      if (subscribed_topic.second.populate_min_subscriber_topic_version) {
+        deltas_needing_min_version.push_back(&topic_delta);
+      }
     }
   }
 
   // Fill in the min subscriber topic version. This must be done after releasing
   // topics_map_lock_.
-  lock_guard<mutex> l(subscribers_lock_);
-  typedef map<TopicId, TTopicDelta> TopicDeltaMap;
-  for (TopicDeltaMap::value_type& topic_delta: update_state_request->topic_deltas) {
-    topic_delta.second.__set_min_subscriber_topic_version(
-        GetMinSubscriberTopicVersion(topic_delta.first));
+  if (!deltas_needing_min_version.empty()) {
+    lock_guard<mutex> l(subscribers_lock_);
+    for (TTopicDelta* delta : deltas_needing_min_version) {
+      delta->__set_min_subscriber_topic_version(
+          GetMinSubscriberTopicVersion(delta->topic_name));
+    }
   }
 }
 
diff --git a/be/src/statestore/statestore.h b/be/src/statestore/statestore.h
index edaef49..71e1ade 100644
--- a/be/src/statestore/statestore.h
+++ b/be/src/statestore/statestore.h
@@ -331,11 +331,17 @@ class Statestore : public CacheLineAligned {
 
     /// Information about a subscriber's subscription to a specific topic.
     struct TopicSubscription {
-      TopicSubscription(bool is_transient) : is_transient(is_transient) {}
+      TopicSubscription(bool is_transient, bool populate_min_subscriber_topic_version)
+        : is_transient(is_transient),
+          populate_min_subscriber_topic_version(populate_min_subscriber_topic_version) {}
 
       /// Whether entries written by this subscriber should be considered transient.
       const bool is_transient;
 
+      /// Whether min_subscriber_topic_version needs to be filled in for this
+      /// subscription.
+      const bool populate_min_subscriber_topic_version;
+
       /// The last topic entry version successfully processed by this subscriber. Only
       /// written by a single thread at a time but can be read concurrently.
       AtomicInt64 last_version{TOPIC_INITIAL_VERSION};
diff --git a/common/thrift/StatestoreService.thrift b/common/thrift/StatestoreService.thrift
index 4f2dada..783bea7 100644
--- a/common/thrift/StatestoreService.thrift
+++ b/common/thrift/StatestoreService.thrift
@@ -139,6 +139,12 @@ struct TTopicRegistration {
   // True if updates to this topic from this subscriber should be removed upon the
   // subscriber's failure or disconnection
   2: required bool is_transient;
+
+  // If true, min_subscriber_topic_version is computed and set in topic updates sent
+  // to this subscriber to this subscriber. Should only be set to true if this is
+  // actually required - computing the version is relatively expensive compared to
+  // other aspects of preparing topic updates - see IMPALA-6816.
+  3: required bool populate_min_subscriber_topic_version = false;
 }
 
 struct TRegisterSubscriberRequest {
diff --git a/tests/statestore/test_statestore.py b/tests/statestore/test_statestore.py
index 2a7c2f9..698a2a6 100644
--- a/tests/statestore/test_statestore.py
+++ b/tests/statestore/test_statestore.py
@@ -590,3 +590,96 @@ class TestStatestore():
     sub.register(topics=[reg])
     LOG.info("Re-registered with id {0}, waiting for update".format(sub.subscriber_id))
     sub.wait_for_update(topic_name, target_updates)
+
+  def test_min_subscriber_topic_version(self):
+    self._do_test_min_subscriber_topic_version(False)
+
+  def test_min_subscriber_topic_version_with_straggler(self):
+    self._do_test_min_subscriber_topic_version(True)
+
+  def _do_test_min_subscriber_topic_version(self, simulate_straggler):
+    """Implementation of test that the 'min_subscriber_topic_version' flag is correctly
+    set when requested. This tests runs two subscribers concurrently and tracks the
+    minimum version each has processed. If 'simulate_straggler' is true, one subscriber
+    rejects updates so that its version is not advanced."""
+    topic_name = "test_min_subscriber_topic_version_%s" % uuid.uuid1()
+
+    # This lock is held while processing the update to protect last_to_versions.
+    update_lock = threading.Lock()
+    last_to_versions = {}
+    TOTAL_SUBSCRIBERS = 2
+    def callback(sub, args, is_producer, sub_name):
+      """Callback for subscriber to verify min_subscriber_topic_version behaviour.
+      If 'is_producer' is true, this acts as the producer, otherwise it acts as the
+      consumer. 'sub_name' is a name used to index into last_to_versions."""
+      if topic_name not in args.topic_deltas:
+        # The update doesn't contain our topic.
+        pass
+      with update_lock:
+        LOG.info("{0} got update {1}".format(sub_name,
+            repr(args.topic_deltas[topic_name])))
+        LOG.info("Versions: {0}".format(last_to_versions))
+        to_version = args.topic_deltas[topic_name].to_version
+        from_version = args.topic_deltas[topic_name].from_version
+        min_subscriber_topic_version = \
+            args.topic_deltas[topic_name].min_subscriber_topic_version
+
+        if is_producer:
+          assert min_subscriber_topic_version is not None
+          assert (to_version == 0 and min_subscriber_topic_version == 0) or\
+              min_subscriber_topic_version < to_version,\
+              "'to_version' hasn't been created yet by this subscriber."
+          # Only validate version once all subscribers have processed an update.
+          if len(last_to_versions) == TOTAL_SUBSCRIBERS:
+            min_to_version = min(last_to_versions.values())
+            assert min_subscriber_topic_version <= min_to_version,\
+                "The minimum subscriber topic version seen by the producer cannot get " +\
+                "ahead of the minimum version seem by the consumer, by definition."
+            assert min_subscriber_topic_version >= min_to_version - 2,\
+                "The min topic version can be two behind the last version seen by " + \
+                "this subscriber because the updates for both subscribers are " + \
+                "prepared in parallel and because it's possible that the producer " + \
+                "processes two updates in-between consumer updates. This is not " + \
+                "absolute but depends on updates not being delayed a large amount."
+        else:
+          # Consumer did not request topic version.
+          assert min_subscriber_topic_version is None
+
+        # Check the 'to_version' and update 'last_to_versions'.
+        last_to_version = last_to_versions.get(sub_name, 0)
+        if to_version > 0:
+          # Non-empty update.
+          assert from_version == last_to_version
+        # Stragglers should accept the first update then skip later ones.
+        skip_update = simulate_straggler and not is_producer and last_to_version > 0
+        if not skip_update: last_to_versions[sub_name] = to_version
+
+        if is_producer:
+          delta = self.make_topic_update(topic_name)
+          return TUpdateStateResponse(status=STATUS_OK, topic_updates=[delta],
+                                      skipped=False)
+        elif skip_update:
+          return TUpdateStateResponse(status=STATUS_OK, topic_updates=[], skipped=True)
+        else:
+          return DEFAULT_UPDATE_STATE_RESPONSE
+
+    # Two concurrent subscribers, which pushes out updates and checks the minimum
+    # version, the other which just consumes the updates.
+    def producer_callback(sub, args): return callback(sub, args, True, "producer")
+    def consumer_callback(sub, args): return callback(sub, args, False, "consumer")
+    consumer_sub = StatestoreSubscriber(update_cb=consumer_callback)
+    consumer_reg = TTopicRegistration(topic_name=topic_name, is_transient=True)
+    producer_sub = StatestoreSubscriber(update_cb=producer_callback)
+    producer_reg = TTopicRegistration(topic_name=topic_name, is_transient=True,
+        populate_min_subscriber_topic_version=True)
+    NUM_UPDATES = 6
+    (
+      consumer_sub.start()
+          .register(topics=[consumer_reg])
+    )
+    (
+      producer_sub.start()
+          .register(topics=[producer_reg])
+          .wait_for_update(topic_name, NUM_UPDATES)
+    )
+    consumer_sub.wait_for_update(topic_name, NUM_UPDATES)


[impala] 12/14: IMPALA-4848: Add WIDTH_BUCKET() function

Posted by ta...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 16276f4e1f55d973709f6c568a673aff6f308b7f
Author: aphadke <ap...@cloudera.com>
AuthorDate: Wed Feb 15 14:09:51 2017 -0800

    IMPALA-4848: Add WIDTH_BUCKET() function
    
    Syntax :
    width_bucket(expr decimal, min_val decimal, max_val decimal,
      num_buckets int)
    
    This function creates equiwidth histograms , where the histogram range
    is divided into num_buckets buckets having identical sizes. This
    function returns the bucket in which the expr value would fall. min_val
    and max_val are the minimum and maximum value of the histogram range
    respectively.
    
    -> This function returns NULL if expr is a NULL.
    -> This function returns 0 if expr < min_val
    -> This function returns num_buckets + 1 if expr > max_val
    
     E.g.
    [localhost:21000] > select width_bucket(8, 1, 20, 3);
    +---------------------------+
    | width_bucket(8, 1, 20, 3) |
    +---------------------------+
    | 2                         |
    +---------------------------+
    
    Change-Id: I081bc916b1bef7b929ca161a9aade3b54c6b858f
    Reviewed-on: http://gerrit.cloudera.org:8080/6023
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/exprs/expr-test.cc                          |  65 +++++++++
 be/src/exprs/math-functions-ir.cc                  | 156 ++++++++++++++++++++-
 be/src/exprs/math-functions.h                      |  15 +-
 be/src/util/string-util.cc                         |   8 ++
 be/src/util/string-util.h                          |   4 +
 common/function-registry/impala_functions.py       |   3 +-
 .../main/java/org/apache/impala/analysis/Expr.java |   2 +-
 7 files changed, 246 insertions(+), 7 deletions(-)

diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index a4ddf15..f76f098 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -59,6 +59,7 @@
 #include "udf/udf-test-harness.h"
 #include "util/debug-util.h"
 #include "util/string-parser.h"
+#include "util/string-util.h"
 #include "util/test-info.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 
@@ -584,6 +585,17 @@ class ExprTest : public testing::Test {
     ASSERT_FALSE(status.ok()) << "stmt: " << stmt << "\nunexpected Status::OK.";
   }
 
+  // "Execute 'expr' and check that the returned error ends with 'error_string'"
+  void TestErrorString(const string& expr, const string& error_string) {
+    string stmt = "select " + expr;
+    vector<FieldSchema> result_types;
+    string result_row;
+    Status status = executor_->Exec(stmt, &result_types);
+    status = executor_->FetchResult(&result_row);
+    ASSERT_FALSE(status.ok());
+    ASSERT_TRUE(EndsWith(status.msg().msg(), error_string));
+  }
+
   template <typename T> void TestFixedPointComparisons(bool test_boundaries) {
     int64_t t_min = numeric_limits<T>::min();
     int64_t t_max = numeric_limits<T>::max();
@@ -5301,6 +5313,59 @@ TEST_F(ExprTest, MathFunctions) {
   TestValue("sqrt(2.0)", TYPE_DOUBLE, sqrt(2.0));
   TestValue("dsqrt(81.0)", TYPE_DOUBLE, 9);
 
+  TestValue("width_bucket(6.3, 2, 17, 2)", TYPE_BIGINT, 1);
+  TestValue("width_bucket(11, 6, 14, 3)", TYPE_BIGINT, 2);
+  TestValue("width_bucket(-1, -5, 5, 3)", TYPE_BIGINT, 2);
+  TestValue("width_bucket(1, -5, 5, 3)", TYPE_BIGINT, 2);
+  TestValue("width_bucket(3, 5, 20.1, 4)", TYPE_BIGINT, 0);
+  TestIsNull("width_bucket(NULL, 5, 20.1, 4)", TYPE_BIGINT);
+  TestIsNull("width_bucket(22, NULL, 20.1, 4)", TYPE_BIGINT);
+  TestIsNull("width_bucket(22, 5, NULL, 4)", TYPE_BIGINT);
+  TestIsNull("width_bucket(22, 5, 20.1, NULL)", TYPE_BIGINT);
+
+  TestValue("width_bucket(22, 5, 20.1, 4)", TYPE_BIGINT, 5);
+  // Test when the result (bucket number) is greater than the max value that can be
+  // stored in a IntVal
+  TestValue("width_bucket(22, 5, 20.1, 2147483647)", TYPE_BIGINT, 2147483648);
+  // Test when min and max of the bucket width range are equal.
+  TestErrorString("width_bucket(22, 5, 5, 4)",
+      "UDF ERROR: Lower bound cannot be greater than or equal to the upper bound\n");
+  // Test when min > max
+  TestErrorString("width_bucket(22, 50, 5, 4)",
+      "UDF ERROR: Lower bound cannot be greater than or equal to the upper bound\n");
+  // Test max - min will overflow during width_bucket evaluation
+  TestErrorString("width_bucket(11, -9, 99999999999999999999999999999999999999, 4000)",
+      "UDF ERROR: Overflow while evaluating the difference between min_range: -9 and "
+      "max_range: 99999999999999999999999999999999999999\n");
+  // If expr - min overflows during width_bucket evaluation, max - min will also
+  // overflow. Since we evaluate max - min before evaluating expr - min, we will never
+  // end up overflowing expr - min.
+  TestErrorString("width_bucket(1, -99999999999999999999999999999999999999, 9, 40)",
+      "UDF ERROR: Overflow while evaluating the difference between min_range: "
+      "-99999999999999999999999999999999999999 and max_range: 9\n");
+  // Test when dist_from_min * buckets cannot be stored in a int128_t (overflows)
+  // and needs to be stored in a int256_t
+  TestValue("width_bucket(8000000000000000000000000000000000000,"
+      "100000000000000000000000000000000000, 9000000000000000000000000000000000000,"
+      "900000)", TYPE_BIGINT, 798877);
+  // Test when range_size * GetScaleMultiplier(input_scale) cannot be stored in a
+  // int128_t (overflows) and needs to be stored in a int256_t
+  TestValue("width_bucket(100000000, 199999.77777777777777777777777777, 99999999999.99999"
+    ", 40)", TYPE_BIGINT, 1);
+  // Test with max values for expr and num_bucket when the width_bucket can be
+  // evaluated with int128_t. Incrementing one of them will require using int256_t for
+  // width_bucket evaluation
+  TestValue("width_bucket(9999999999999999999999999999999999999, 1,"
+            "99999999999999999999999999999999999999, 15)", TYPE_BIGINT, 2);
+  // Test with the smallest value of num_bucket for the given combination of expr,
+  // max and min value that would require int256_t for evalation
+  TestValue("width_bucket(9999999999999999999999999999999999999, 1,"
+            "99999999999999999999999999999999999999, 16)", TYPE_BIGINT, 2);
+  // Test with the smallest value of expr for the given combination of num_buckets,
+  // max and min value that would require int256_t for evalation
+  TestValue("width_bucket(10000000000000000000000000000000000000, 1,"
+            "99999999999999999999999999999999999999, 15)", TYPE_BIGINT, 2);
+
   // Run twice to test deterministic behavior.
   for (uint32_t seed : {0, 1234}) {
     stringstream rand, random;
diff --git a/be/src/exprs/math-functions-ir.cc b/be/src/exprs/math-functions-ir.cc
index 531bf24..6bfb672 100644
--- a/be/src/exprs/math-functions-ir.cc
+++ b/be/src/exprs/math-functions-ir.cc
@@ -15,21 +15,22 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "exprs/math-functions.h"
-
 #include <iomanip>
 #include <random>
 #include <sstream>
 #include <math.h>
-
+#include <stdint.h>
+#include <string>
 #include "exprs/anyval-util.h"
 #include "exprs/scalar-expr.h"
 #include "exprs/operators.h"
+#include "exprs/math-functions.h"
 #include "util/string-parser.h"
+#include "runtime/decimal-value.inline.h"
 #include "runtime/runtime-state.h"
 #include "runtime/string-value.inline.h"
 #include "thirdparty/pcg-cpp-0.98/include/pcg_random.hpp"
-
+#include "exprs/decimal-operators.h"
 #include "common/names.h"
 
 using std::uppercase;
@@ -431,6 +432,153 @@ DoubleVal MathFunctions::FmodDouble(FunctionContext* ctx, const DoubleVal& a,
   return DoubleVal(fmod(a.val, b.val));
 }
 
+// The bucket_number is evaluated using the following formula:
+//
+//   bucket_number = dist_from_min * num_buckets / range_size
+//      where -
+//        dist_from_min = expr - min_range
+//        range_size = max_range - min_range
+//        buckets = number of buckets
+//
+// The results of the above subtractions are stored in Decimal16Value to avoid an overflow
+// in the following cases:
+//   case 1:
+//      T1 is decimal8Value
+//         When evaluating this particular expression
+//            dist_from_min = expr - min_range
+//         If expr is a max positive value which can be represented in decimal8Value and
+//         min_range < 0 the resulting dist_from_min can be represented in decimal16Val
+//         without overflowing
+//   case 2:
+//      T1 is decimal16Value
+//         Subtracting a negative min_range from expr can result in an overflow in which
+//         case the function errors out. There is no decimal32Val to handle this. So
+//         storing it in decimal16Value.
+//   case 3:
+//      T1 is decimal4Value
+//         We can store the results in a decimal8Value. But this change hard codes to
+//         store the result in decimal16Val for now to be compatible with the other
+//         decimal*Vals
+//
+// The result of this multiplication dist_from_min * buckets is stored as a int256_t
+// if storing it in a int128_t would overflow.
+//
+// To perform the division, range_size is scaled up. The scale and precision of the
+// numerator and denominator are adjusted to be the same. This avoids the need to compute
+// the resulting scale and precision.
+template <class  T1>
+BigIntVal MathFunctions::WidthBucketImpl(FunctionContext* ctx,
+    const T1& expr, const T1& min_range,
+    const T1& max_range, const IntVal& num_buckets) {
+  // FE casts expr, min_range and max_range to be of the scale and precision
+  int input_scale = ctx->impl()->GetConstFnAttr(FunctionContextImpl::ARG_TYPE_SCALE, 1);
+  int input_precision = ctx->impl()->GetConstFnAttr(
+      FunctionContextImpl::ARG_TYPE_PRECISION, 1);
+
+  bool overflow = false;
+  Decimal16Value range_size = max_range.template Subtract<int128_t>(input_scale,
+      min_range, input_scale, input_precision, input_scale, false, &overflow);
+  if (UNLIKELY(overflow)) {
+    ostringstream error_msg;
+    error_msg << "Overflow while evaluating the difference between min_range: " <<
+        min_range.value() << " and max_range: " << max_range.value();
+    ctx->SetError(error_msg.str().c_str());
+    return BigIntVal::null();
+  }
+
+  if (UNLIKELY(num_buckets.val <= 0)) {
+    ostringstream error_msg;
+    error_msg << "Number of buckets should be greater than zero:" << num_buckets.val;
+    ctx->SetError(error_msg.str().c_str());
+    return BigIntVal::null();
+  }
+
+  if (UNLIKELY(min_range >= max_range)) {
+    ctx->SetError("Lower bound cannot be greater than or equal to the upper bound");
+    return BigIntVal::null();
+  }
+
+  if (UNLIKELY(expr < min_range)) return 0;
+
+  if (UNLIKELY(expr >= max_range)) {
+    BigIntVal result;
+    result.val = num_buckets.val;
+    ++result.val;
+    return result;
+  }
+
+  Decimal16Value dist_from_min = expr.template Subtract<int128_t>(input_scale,
+      min_range, input_scale, input_precision, input_scale, false, &overflow);
+  DCHECK_EQ(overflow, false);
+
+  Decimal16Value buckets = Decimal16Value::FromInt(input_precision, input_scale,
+      num_buckets.val, &overflow);
+
+  bool needs_int256 = false;
+  // Check if dist_from_min * buckets would overflow and if there is a need to
+  // store the intermediate results in int256_t to avoid an overflows
+  // Check if scaling up range size overflows and if there is a need to store the
+  // intermediate results in int256_t to avoid the overflow
+  if (UNLIKELY(BitUtil::CountLeadingZeros(abs(buckets.value())) +
+      BitUtil::CountLeadingZeros(abs(dist_from_min.value())) <= 128 ||
+      BitUtil::CountLeadingZeros(range_size.value()) +
+      detail::MinLeadingZerosAfterScaling(BitUtil::CountLeadingZeros(
+      range_size.value()), input_scale) <= 128)) {
+    needs_int256 = true;
+  }
+
+  int128_t result;
+  if (needs_int256) {
+    // resulting scale should be 2 * input_scale as per multiplication rules
+    int256_t x = ConvertToInt256(buckets.value()) * ConvertToInt256(
+        dist_from_min.value());
+
+    // Since "range_size" and "x" have different scales, the divison would require
+    // evaluating the resulting scale. To avoid this we scale up the denominator to
+    // match the scale of the numerator.
+    int256_t y = DecimalUtil::MultiplyByScale<int256_t>(ConvertToInt256(
+        range_size.value()), input_scale, false);
+    result = ConvertToInt128(x / y, DecimalUtil::MAX_UNSCALED_DECIMAL16,
+        &overflow);
+    DCHECK_EQ(overflow, false);
+  } else {
+    // resulting scale should be 2 * input_scale as per multiplication rules
+    int128_t x = buckets.value() * dist_from_min.value();
+
+    // Since "range_size" and "x" have different scales, the divison would require
+    // evaluating the resulting scale. To avoid this we scale up the denominator to
+    // match the scale of the numerator.
+    int128_t y = DecimalUtil::MultiplyByScale<int128_t>(range_size.value(),
+        input_scale, false);
+    result = x / y; // NOLINT: clang-tidy thinks y may equal zero here.
+  }
+  return (BigIntVal(abs(result) + 1));
+}
+
+BigIntVal MathFunctions::WidthBucket(FunctionContext* ctx, const DecimalVal& expr,
+    const DecimalVal& min_range, const DecimalVal& max_range,
+    const IntVal& num_buckets) {
+  if (expr.is_null || min_range.is_null || max_range.is_null || num_buckets.is_null) {
+    return BigIntVal::null();
+  }
+
+  int arg_size_type = ctx->impl()->GetConstFnAttr(FunctionContextImpl::ARG_TYPE_SIZE, 0);
+  switch (arg_size_type) {
+    case 4:
+      return WidthBucketImpl<Decimal4Value> (ctx, Decimal4Value(expr.val4),
+          Decimal4Value(min_range.val4), Decimal4Value(max_range.val4), num_buckets);
+    case 8:
+      return WidthBucketImpl<Decimal8Value> (ctx, Decimal8Value(expr.val8),
+          Decimal8Value(min_range.val8), Decimal8Value(max_range.val8), num_buckets);
+    case 16:
+      return WidthBucketImpl<Decimal16Value>(ctx, Decimal16Value(expr.val16),
+         Decimal16Value(min_range.val16), Decimal16Value(max_range.val16), num_buckets);
+    default:
+      DCHECK(false);
+      return BigIntVal::null();
+  }
+}
+
 template <typename T> T MathFunctions::Positive(FunctionContext* ctx, const T& val) {
   return val;
 }
diff --git a/be/src/exprs/math-functions.h b/be/src/exprs/math-functions.h
index 7063907..e9f5c89 100644
--- a/be/src/exprs/math-functions.h
+++ b/be/src/exprs/math-functions.h
@@ -15,7 +15,6 @@
 // specific language governing permissions and limitations
 // under the License.
 
-
 #ifndef IMPALA_EXPRS_MATH_FUNCTIONS_H
 #define IMPALA_EXPRS_MATH_FUNCTIONS_H
 
@@ -112,6 +111,11 @@ class MathFunctions {
   template <bool ISLEAST> static DecimalVal LeastGreatest(
       FunctionContext*, int num_args, const DecimalVal* args);
 
+
+  static BigIntVal WidthBucket(FunctionContext* ctx, const DecimalVal& expr,
+      const DecimalVal& min_range, const DecimalVal& max_range,
+      const IntVal& num_buckets);
+
  private:
   static const int32_t MIN_BASE = 2;
   static const int32_t MAX_BASE = 36;
@@ -135,6 +139,15 @@ class MathFunctions {
   /// Returns false otherwise, indicating some other error condition.
   static bool HandleParseResult(int8_t dest_base, int64_t* num,
       StringParser::ParseResult parse_res);
+
+  /// This function creates equiwidth histograms , where the histogram range
+  /// is divided into num_buckets buckets having identical sizes. This function
+  /// returns the bucket in which the expr value would fall. min_val and
+  /// max_val are the minimum and maximum value of the histogram range
+  /// respectively.
+  template <typename T1>
+  static BigIntVal WidthBucketImpl(FunctionContext* ctx,const T1& expr,
+      const T1& min_range,const T1& max_range, const IntVal& num_buckets);
 };
 
 }
diff --git a/be/src/util/string-util.cc b/be/src/util/string-util.cc
index f8e284f..e442391 100644
--- a/be/src/util/string-util.cc
+++ b/be/src/util/string-util.cc
@@ -65,4 +65,12 @@ bool CommaSeparatedContains(const std::string& cs_list, const std::string& item)
   return false;
 }
 
+bool EndsWith(const std::string& full_string, const std::string& end) {
+  if (full_string.size() >= end.size()) {
+    return (full_string.compare(full_string.size() - end.size(), end.size(),
+        end) == 0);
+  }
+  return false;
+}
+
 }
diff --git a/be/src/util/string-util.h b/be/src/util/string-util.h
index 2811a3c..8db010c 100644
--- a/be/src/util/string-util.h
+++ b/be/src/util/string-util.h
@@ -40,6 +40,10 @@ Status TruncateUp(const std::string& str, int32_t max_length, std::string* resul
 /// Return true if the comma-separated string 'cs_list' contains 'item' as one of
 /// the comma-separated values.
 bool CommaSeparatedContains(const std::string& cs_list, const std::string& item);
+
+/// Return true if a given string 'full_str' ends with the characters in the
+/// 'end' string
+bool EndsWith(const std::string& full_string, const std::string& end);
 }
 
 #endif
diff --git a/common/function-registry/impala_functions.py b/common/function-registry/impala_functions.py
index b3387c6..1aba496 100644
--- a/common/function-registry/impala_functions.py
+++ b/common/function-registry/impala_functions.py
@@ -388,7 +388,8 @@ visible_functions = [
    '_ZN6impala13MathFunctions13LeastGreatestILb0EEEN10impala_udf9StringValEPNS2_15FunctionContextEiPKS3_'],
   [['greatest'], 'DECIMAL', ['DECIMAL', '...'],
    '_ZN6impala13MathFunctions13LeastGreatestILb0EEEN10impala_udf10DecimalValEPNS2_15FunctionContextEiPKS3_'],
-
+  [['width_bucket'], 'BIGINT', ['DECIMAL', 'DECIMAL', 'DECIMAL', 'INT'],
+    '_ZN6impala13MathFunctions11WidthBucketEPN10impala_udf15FunctionContextERKNS1_10DecimalValES6_S6_RKNS1_6IntValE'],
   # Decimal Functions
   # TODO: oracle has decimal support for transcendental functions (e.g. sin()) to very
   # high precisions. Do we need them? It's unclear if other databases do the same.
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 5cdcb16..e85a4a4 100644
--- a/fe/src/main/java/org/apache/impala/analysis/Expr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/Expr.java
@@ -504,12 +504,12 @@ abstract public class Expr extends TreeNode<Expr> implements ParseNode, Cloneabl
         ScalarType decimalType = (ScalarType) childType;
         result = decimalType.getMinResolutionDecimal();
       } else {
-        Preconditions.checkState(childType.isDecimal() || result.isDecimal());
         result = Type.getAssignmentCompatibleType(
             result, childType, false, strictDecimal);
       }
     }
     if (result != null && !result.isNull()) {
+      result = ((ScalarType)result).getMinResolutionDecimal();
       Preconditions.checkState(result.isDecimal() || result.isInvalid());
       Preconditions.checkState(!result.isWildcardDecimal());
     }