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 2018/02/23 22:51:28 UTC

[01/15] impala git commit: IMPALA-4835: Part 3: switch I/O buffers to buffer pool

Repository: impala
Updated Branches:
  refs/heads/2.x e0c09181f -> 8c922a6ef


http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/tpch/queries/sort-reservation-usage.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/tpch/queries/sort-reservation-usage.test b/testdata/workloads/tpch/queries/sort-reservation-usage.test
index 92f180d..af31ccb 100644
--- a/testdata/workloads/tpch/queries/sort-reservation-usage.test
+++ b/testdata/workloads/tpch/queries/sort-reservation-usage.test
@@ -1,17 +1,18 @@
 ====
 ---- QUERY
 # Test that in-mem sorts incrementally give up memory when emitting output.
-# This query and the limit is calibrated to fail if the first sort does not
-# give up memory to the second sort.
+# This query and scratch limit are chosen so that the query fails if the first sort
+# does not give up memory to the second sort. Scans the text formatted file so that
+# the scan uses less reservation.
 set num_nodes=1;
 set scratch_limit=0;
-set buffer_pool_limit=15m;
+set buffer_pool_limit=35m;
 set default_spillable_buffer_size=64kb;
 SELECT *
 FROM   (SELECT
         Rank() OVER(ORDER BY  l_orderkey) AS rank,
         Rank() OVER(ORDER BY  l_partkey) AS rank2
-        FROM lineitem
+        FROM tpch.lineitem
         WHERE l_shipdate < '1992-05-09') a
 WHERE rank < 10
 ORDER BY rank;

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/tests/common/test_dimensions.py
----------------------------------------------------------------------
diff --git a/tests/common/test_dimensions.py b/tests/common/test_dimensions.py
index 4171e1f..3c1b5e7 100644
--- a/tests/common/test_dimensions.py
+++ b/tests/common/test_dimensions.py
@@ -131,13 +131,13 @@ SINGLE_NODE_ONLY = [1]
 ALL_NODES_ONLY = [0]
 ALL_DISABLE_CODEGEN_OPTIONS = [True, False]
 
-def create_single_exec_option_dimension():
+def create_single_exec_option_dimension(num_nodes=0, disable_codegen_rows_threshold=5000):
   """Creates an exec_option dimension that will produce a single test vector"""
-  return create_exec_option_dimension(cluster_sizes=ALL_NODES_ONLY,
-                                      disable_codegen_options=[False],
-                                      # Make sure codegen kicks in for functional.alltypes.
-                                      disable_codegen_rows_threshold_options=[5000],
-                                      batch_sizes=[0])
+  return create_exec_option_dimension(cluster_sizes=[num_nodes],
+      disable_codegen_options=[False],
+      # Make sure codegen kicks in for functional.alltypes.
+      disable_codegen_rows_threshold_options=[disable_codegen_rows_threshold],
+      batch_sizes=[0])
 
 def create_exec_option_dimension(cluster_sizes=ALL_CLUSTER_SIZES,
                                  disable_codegen_options=ALL_DISABLE_CODEGEN_OPTIONS,
@@ -145,13 +145,15 @@ def create_exec_option_dimension(cluster_sizes=ALL_CLUSTER_SIZES,
                                  sync_ddl=None, exec_single_node_option=[0],
                                  # We already run with codegen on and off explicitly -
                                  # don't need automatic toggling.
-                                 disable_codegen_rows_threshold_options=[0]):
+                                 disable_codegen_rows_threshold_options=[0],
+                                 debug_action_options=[None]):
   exec_option_dimensions = {
       'abort_on_error': [1],
       'exec_single_node_rows_threshold': exec_single_node_option,
       'batch_size': batch_sizes,
       'disable_codegen': disable_codegen_options,
       'disable_codegen_rows_threshold': disable_codegen_rows_threshold_options,
+      'debug_action': debug_action_options,
       'num_nodes': cluster_sizes}
 
   if sync_ddl is not None:

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/tests/custom_cluster/test_scratch_disk.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_scratch_disk.py b/tests/custom_cluster/test_scratch_disk.py
index bd3c7e4..65bde66 100644
--- a/tests/custom_cluster/test_scratch_disk.py
+++ b/tests/custom_cluster/test_scratch_disk.py
@@ -39,7 +39,7 @@ class TestScratchDir(CustomClusterTestSuite):
       """
   # Buffer pool limit that is low enough to force Impala to spill to disk when executing
   # spill_query.
-  buffer_pool_limit = "32m"
+  buffer_pool_limit = "45m"
 
   def count_nonempty_dirs(self, dirs):
     count = 0

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/tests/query_test/test_mem_usage_scaling.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_mem_usage_scaling.py b/tests/query_test/test_mem_usage_scaling.py
index c60b1c5..f13e320 100644
--- a/tests/query_test/test_mem_usage_scaling.py
+++ b/tests/query_test/test_mem_usage_scaling.py
@@ -95,7 +95,7 @@ class TestExprMemUsage(ImpalaTestSuite):
 class TestLowMemoryLimits(ImpalaTestSuite):
   '''Super class for the memory limit tests with the TPC-H and TPC-DS queries'''
 
-  def low_memory_limit_test(self, vector, tpch_query, limit, xfail_mem_limit=None):
+  def low_memory_limit_test(self, vector, tpch_query, limit):
     mem = vector.get_value('mem_limit')
     # Mem consumption can be +-30MBs, depending on how many scanner threads are
     # running. Adding this extra mem in order to reduce false negatives in the tests.
@@ -112,13 +112,11 @@ class TestLowMemoryLimits(ImpalaTestSuite):
     try:
       self.run_test_case(tpch_query, new_vector)
     except ImpalaBeeswaxException as e:
-      if not expects_error and not xfail_mem_limit: raise
+      if not expects_error: raise
       found_expected_error = False
       for error_msg in MEM_LIMIT_ERROR_MSGS:
         if error_msg in str(e): found_expected_error = True
       assert found_expected_error, str(e)
-      if not expects_error and xfail_mem_limit:
-        pytest.xfail(xfail_mem_limit)
 
 
 class TestTpchMemLimitError(TestLowMemoryLimits):
@@ -132,7 +130,7 @@ class TestTpchMemLimitError(TestLowMemoryLimits):
                        'Q6' : 25, 'Q7' : 200, 'Q8' : 125, 'Q9' : 200, 'Q10' : 162,\
                        'Q11' : 112, 'Q12' : 150, 'Q13' : 125, 'Q14' : 125, 'Q15' : 125,\
                        'Q16' : 137, 'Q17' : 137, 'Q18' : 196, 'Q19' : 112, 'Q20' : 162,\
-                       'Q21' : 187, 'Q22' : 125}
+                       'Q21' : 230, 'Q22' : 125}
 
   @classmethod
   def get_workload(self):
@@ -175,8 +173,7 @@ class TestTpchMemLimitError(TestLowMemoryLimits):
     self.low_memory_limit_test(vector, 'tpch-q8', self.MIN_MEM_FOR_TPCH['Q8'])
 
   def test_low_mem_limit_q9(self, vector):
-    self.low_memory_limit_test(vector, 'tpch-q9', self.MIN_MEM_FOR_TPCH['Q9'],
-            xfail_mem_limit="IMPALA-3328: TPC-H Q9 memory limit test is flaky")
+    self.low_memory_limit_test(vector, 'tpch-q9', self.MIN_MEM_FOR_TPCH['Q9'])
 
   @SkipIfLocal.mem_usage_different
   def test_low_mem_limit_q10(self, vector):

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/tests/query_test/test_query_mem_limit.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_query_mem_limit.py b/tests/query_test/test_query_mem_limit.py
index 2fdd6eb..17ea9f5 100644
--- a/tests/query_test/test_query_mem_limit.py
+++ b/tests/query_test/test_query_mem_limit.py
@@ -118,7 +118,9 @@ class TestCodegenMemLimit(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestCodegenMemLimit, cls).add_test_dimensions()
-    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension())
+    # Run with num_nodes=1 to avoid races between fragments allocating memory.
+    cls.ImpalaTestMatrix.add_dimension(create_single_exec_option_dimension(
+        num_nodes=1, disable_codegen_rows_threshold=0))
     # Only run the query for parquet
     cls.ImpalaTestMatrix.add_constraint(
       lambda v: v.get_value('table_format').file_format == 'parquet')

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/tests/query_test/test_scanners.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners.py b/tests/query_test/test_scanners.py
index 9b252da..2d0fd0c 100644
--- a/tests/query_test/test_scanners.py
+++ b/tests/query_test/test_scanners.py
@@ -37,7 +37,9 @@ from tests.common.skip import (
     SkipIfADLS,
     SkipIfIsilon,
     SkipIfLocal)
-from tests.common.test_dimensions import create_single_exec_option_dimension
+from tests.common.test_dimensions import (
+    create_single_exec_option_dimension,
+    create_exec_option_dimension)
 from tests.common.test_result_verifier import (
     parse_column_types,
     parse_column_labels,
@@ -49,6 +51,11 @@ from tests.util.hdfs_util import NAMENODE
 from tests.util.get_parquet_metadata import get_parquet_metadata
 from tests.util.test_file_parser import QueryTestSectionReader
 
+# Test scanners with denial of reservations at varying frequency. This will affect the
+# number of scanner threads that can be spun up.
+DEBUG_ACTION_DIMS = [None,
+  '-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@0.5',
+  '-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0']
 
 class TestScannersAllTableFormats(ImpalaTestSuite):
   BATCH_SIZES = [0, 1, 16]
@@ -66,18 +73,20 @@ class TestScannersAllTableFormats(ImpalaTestSuite):
       cls.ImpalaTestMatrix.add_dimension(cls.create_table_info_dimension('pairwise'))
     cls.ImpalaTestMatrix.add_dimension(
         ImpalaTestDimension('batch_size', *TestScannersAllTableFormats.BATCH_SIZES))
+    cls.ImpalaTestMatrix.add_dimension(
+        ImpalaTestDimension('debug_action', *DEBUG_ACTION_DIMS))
 
   def test_scanners(self, vector):
     new_vector = deepcopy(vector)
     new_vector.get_value('exec_option')['batch_size'] = vector.get_value('batch_size')
+    new_vector.get_value('exec_option')['debug_action'] = vector.get_value('debug_action')
     self.run_test_case('QueryTest/scanners', new_vector)
 
   def test_hdfs_scanner_profile(self, vector):
-    if vector.get_value('table_format').file_format in ('kudu', 'hbase'):
+    if vector.get_value('table_format').file_format in ('kudu', 'hbase') or \
+       vector.get_value('exec_option')['num_nodes'] != 0:
       pytest.skip()
-    new_vector = deepcopy(vector)
-    new_vector.get_value('exec_option')['num_nodes'] = 0
-    self.run_test_case('QueryTest/hdfs_scanner_profile', new_vector)
+    self.run_test_case('QueryTest/hdfs_scanner_profile', vector)
 
 # Test all the scanners with a simple limit clause. The limit clause triggers
 # cancellation in the scanner code paths.
@@ -171,6 +180,8 @@ class TestWideRow(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestWideRow, cls).add_test_dimensions()
+    cls.ImpalaTestMatrix.add_dimension(
+        create_exec_option_dimension(debug_action_options=DEBUG_ACTION_DIMS))
     # I can't figure out how to load a huge row into hbase
     cls.ImpalaTestMatrix.add_constraint(
       lambda v: v.get_value('table_format').file_format != 'hbase')
@@ -202,6 +213,8 @@ class TestWideTable(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestWideTable, cls).add_test_dimensions()
+    cls.ImpalaTestMatrix.add_dimension(
+        create_exec_option_dimension(debug_action_options=DEBUG_ACTION_DIMS))
     cls.ImpalaTestMatrix.add_dimension(ImpalaTestDimension("num_cols", *cls.NUM_COLS))
     # To cut down on test execution time, only run in exhaustive.
     if cls.exploration_strategy() != 'exhaustive':
@@ -244,6 +257,8 @@ class TestParquet(ImpalaTestSuite):
   @classmethod
   def add_test_dimensions(cls):
     super(TestParquet, cls).add_test_dimensions()
+    cls.ImpalaTestMatrix.add_dimension(
+        create_exec_option_dimension(debug_action_options=DEBUG_ACTION_DIMS))
     cls.ImpalaTestMatrix.add_constraint(
       lambda v: v.get_value('table_format').file_format == 'parquet')
 

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/tests/query_test/test_scanners_fuzz.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_scanners_fuzz.py b/tests/query_test/test_scanners_fuzz.py
index c336a17..0b42e5d 100644
--- a/tests/query_test/test_scanners_fuzz.py
+++ b/tests/query_test/test_scanners_fuzz.py
@@ -48,6 +48,12 @@ class TestScannersFuzzing(ImpalaTestSuite):
   # Test a range of batch sizes to exercise different corner cases.
   BATCH_SIZES = [0, 1, 16, 10000]
 
+  # Test with denial of reservations at varying frequency. This will affect the number
+  # of scanner threads that can be spun up.
+  DEBUG_ACTION_VALUES = [None,
+    '-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@0.5',
+    '-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0']
+
   @classmethod
   def get_workload(cls):
     return 'functional-query'
@@ -59,7 +65,8 @@ class TestScannersFuzzing(ImpalaTestSuite):
         create_exec_option_dimension_from_dict({
           'abort_on_error' : cls.ABORT_ON_ERROR_VALUES,
           'num_nodes' : cls.NUM_NODES_VALUES,
-          'mem_limit' : cls.MEM_LIMITS}))
+          'mem_limit' : cls.MEM_LIMITS,
+          'debug_action' : cls.DEBUG_ACTION_VALUES}))
     # TODO: enable for more table formats once they consistently pass the fuzz test.
     cls.ImpalaTestMatrix.add_constraint(lambda v:
         v.get_value('table_format').file_format in ('avro', 'parquet') or

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/tests/query_test/test_sort.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_sort.py b/tests/query_test/test_sort.py
index 59a28cd..70629c4 100644
--- a/tests/query_test/test_sort.py
+++ b/tests/query_test/test_sort.py
@@ -67,14 +67,23 @@ class TestQueryFullSort(ImpalaTestSuite):
       order by o_orderdate"""
     exec_option = copy(vector.get_value('exec_option'))
     table_format = vector.get_value('table_format')
+    exec_option['default_spillable_buffer_size'] = '8M'
+
+    # Minimum memory for different parts of the plan.
+    sort_reservation_mb = 48
+    if table_format.file_format == 'parquet':
+      scan_reservation_mb = 24
+    else:
+      scan_reservation_mb = 8
+    total_reservation_mb = sort_reservation_mb + scan_reservation_mb
 
     # The below memory value assume 8M pages.
-    exec_option['default_spillable_buffer_size'] = '8M'
-    buffer_pool_limit_values = ['-1', '48M'] # Unlimited and minimum memory.
+    # Test with unlimited and minimum memory for all file formats.
+    buffer_pool_limit_values = ['-1', '{0}M'.format(total_reservation_mb)]
     if self.exploration_strategy() == 'exhaustive' and \
         table_format.file_format == 'parquet':
       # Test some intermediate values for parquet on exhaustive.
-      buffer_pool_limit_values += ['64M', '128M', '256M']
+      buffer_pool_limit_values += ['128M', '256M']
     for buffer_pool_limit in buffer_pool_limit_values:
       exec_option['buffer_pool_limit'] = buffer_pool_limit
       result = transpose_results(self.execute_query(
@@ -83,7 +92,6 @@ class TestQueryFullSort(ImpalaTestSuite):
 
   def test_sort_join(self, vector):
     """With 200m memory limit this should be a 2-phase sort"""
-
     query = """select o1.o_orderdate, o2.o_custkey, o1.o_comment from orders o1 join
     orders o2 on (o1.o_orderkey = o2.o_orderkey) order by o1.o_orderdate limit 100000"""
 

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/tests/query_test/test_spilling.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_spilling.py b/tests/query_test/test_spilling.py
index 0b36429..05ada9f 100644
--- a/tests/query_test/test_spilling.py
+++ b/tests/query_test/test_spilling.py
@@ -95,3 +95,8 @@ class TestSpillingNoDebugActionDimensions(ImpalaTestSuite):
       setting debug_action to alternative values via query options."""
     self.run_test_case('QueryTest/spilling-query-options', vector)
 
+  def test_spilling_no_debug_action(self, vector):
+    """Spilling tests that will not succeed if run with an arbitrary debug action.
+       These tests either run with no debug action set or set their own debug action."""
+    self.run_test_case('QueryTest/spilling-no-debug-action', vector)
+


[15/15] impala git commit: IMPALA-6008: Creating a UDF from a shared library with a .ll extenion crashes impala

Posted by ta...@apache.org.
IMPALA-6008: Creating a UDF from a shared library with a .ll extenion
crashes impala

Impala crashes on creating a UDF from a shared library (.so file) which
was renamed to have .ll extension. CreateFile() call in GetSymbols()
fails and returns on error and does not close the codegen object. This
patch closes the codegen object on failure. This avoids hitting a DCHECK
later up in the stack.
The chain of failures also invokes the DiagnosticHandlerFn. RuntimeState
object is NULL when the DiagnosticHandlerFn gets called in this case.
This change also adds a check before accessing it for logging.

[localhost:21000] > create function foo4 (string, string) returns string
location '/tmp/bad_udf.ll' symbol='MyAwesomeUdf';
Query: create function foo4 (string, string) returns string location
'/tmp/bad_udf.ll' symbol='MyAwesomeUdf'
ERROR: AnalysisException: Could not load binary: /tmp/bad_udf.ll
LLVM diagnostic error: Invalid bitcode signature

Change-Id: Id060668802ca9c80367cdc0e8a823b968d549bbb
Reviewed-on: http://gerrit.cloudera.org:8080/9154
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins
Reviewed-on: http://gerrit.cloudera.org:8080/9424
Tested-by: Tim Armstrong <ta...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/7c4689c8
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/7c4689c8
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/7c4689c8

Branch: refs/heads/2.x
Commit: 7c4689c81f10f50d07aee6ac3057bda3d64d2403
Parents: 84fffd4
Author: aphadke <ap...@cloudera.com>
Authored: Mon Jan 29 11:50:33 2018 -0800
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Fri Feb 23 22:51:01 2018 +0000

----------------------------------------------------------------------
 be/src/codegen/llvm-codegen.cc                  | 34 +++++++++++++-------
 .../queries/QueryTest/udf-errors.test           |  6 ++++
 tests/query_test/test_udfs.py                   | 22 +++++++++++--
 3 files changed, 48 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/7c4689c8/be/src/codegen/llvm-codegen.cc
----------------------------------------------------------------------
diff --git a/be/src/codegen/llvm-codegen.cc b/be/src/codegen/llvm-codegen.cc
index 892b395..3796d76 100644
--- a/be/src/codegen/llvm-codegen.cc
+++ b/be/src/codegen/llvm-codegen.cc
@@ -215,9 +215,14 @@ Status LlvmCodeGen::CreateFromFile(RuntimeState* state, ObjectPool* pool,
   SCOPED_TIMER((*codegen)->profile_->total_time_counter());
 
   unique_ptr<llvm::Module> loaded_module;
-  RETURN_IF_ERROR((*codegen)->LoadModuleFromFile(file, &loaded_module));
-
-  return (*codegen)->Init(std::move(loaded_module));
+  Status status = (*codegen)->LoadModuleFromFile(file, &loaded_module);
+  if (!status.ok()) goto error;
+  status = (*codegen)->Init(std::move(loaded_module));
+  if (!status.ok()) goto error;
+  return Status::OK();
+error:
+  (*codegen)->Close();
+  return status;
 }
 
 Status LlvmCodeGen::CreateFromMemory(RuntimeState* state, ObjectPool* pool,
@@ -242,9 +247,15 @@ Status LlvmCodeGen::CreateFromMemory(RuntimeState* state, ObjectPool* pool,
   unique_ptr<llvm::MemoryBuffer> module_ir_buf(
       llvm::MemoryBuffer::getMemBuffer(module_ir, "", false));
   unique_ptr<llvm::Module> loaded_module;
-  RETURN_IF_ERROR((*codegen)->LoadModuleFromMemory(std::move(module_ir_buf),
-      module_name, &loaded_module));
-  return (*codegen)->Init(std::move(loaded_module));
+  Status status = (*codegen)->LoadModuleFromMemory(std::move(module_ir_buf),
+      module_name, &loaded_module);
+  if (!status.ok()) goto error;
+  status = (*codegen)->Init(std::move(loaded_module));
+  if (!status.ok()) goto error;
+  return Status::OK();
+error:
+  (*codegen)->Close();
+  return status;
 }
 
 Status LlvmCodeGen::LoadModuleFromFile(
@@ -276,9 +287,8 @@ Status LlvmCodeGen::LoadModuleFromMemory(unique_ptr<llvm::MemoryBuffer> module_i
   llvm::ErrorOr<unique_ptr<llvm::Module>> tmp_module =
       getLazyBitcodeModule(std::move(module_ir_buf), context(), false);
   if (!tmp_module) {
-    stringstream ss;
-    ss << "Could not parse module " << module_name << ": " << tmp_module.getError();
-    return Status(ss.str());
+    string diagnostic_err = diagnostic_handler_.GetErrorString();
+    return Status(diagnostic_err);
   }
 
   *module = std::move(tmp_module.get());
@@ -1690,8 +1700,10 @@ void LlvmCodeGen::DiagnosticHandler::DiagnosticHandlerFn(
     diagnostic_printer << "LLVM diagnostic error: ";
     info.print(diagnostic_printer);
     error_msg.flush();
-    LOG(INFO) << "Query " << codegen->state_->query_id() << " encountered a "
-        << codegen->diagnostic_handler_.error_str_;
+    if (codegen->state_) {
+      LOG(INFO) << "Query " << codegen->state_->query_id() << " encountered a "
+          << codegen->diagnostic_handler_.error_str_;
+    }
   }
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/7c4689c8/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
index 9698717..81e459a 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/udf-errors.test
@@ -20,6 +20,12 @@ symbol='FnDoesNotExist';
 Could not load binary: $FILESYSTEM_PREFIX/test-warehouse/not-a-real-file.so
 ====
 ---- QUERY
+create function if not exists foo (string, string) returns string location
+'$FILESYSTEM_PREFIX/test-warehouse/$DATABASE_bad_udf.ll' symbol='MyAwesomeUdf';
+---- CATCH
+Could not load binary: $FILESYSTEM_PREFIX/test-warehouse/$DATABASE_bad_udf.ll
+====
+---- QUERY
 # This test is run with codegen disabled. Interpretation only handles up to 20 arguments.
 create function if not exists twenty_args(int, int, int, int, int, int,
     int, int, int, int, int, int, int, int, int, int, int, int, int, int) returns int

http://git-wip-us.apache.org/repos/asf/impala/blob/7c4689c8/tests/query_test/test_udfs.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_udfs.py b/tests/query_test/test_udfs.py
index 1ff716a..1be25e5 100644
--- a/tests/query_test/test_udfs.py
+++ b/tests/query_test/test_udfs.py
@@ -18,7 +18,11 @@
 from copy import copy
 import os
 import pytest
-from subprocess import check_call
+import random
+import threading
+import time
+import tempfile
+from subprocess import call, check_call
 
 from tests.beeswax.impala_beeswax import ImpalaBeeswaxException
 from tests.common.impala_cluster import ImpalaCluster
@@ -327,8 +331,20 @@ class TestUdfExecution(TestUdfBase):
     # Aim to exercise two failure cases:
     # 1. too many arguments
     # 2. IR UDF
-    if vector.get_value('exec_option')['disable_codegen']:
-      self.run_test_case('QueryTest/udf-errors', vector, use_db=unique_database)
+    fd, dir_name = tempfile.mkstemp()
+    try:
+      with open(dir_name, "w") as f:
+        f.write("Hello World")
+      check_call(["hadoop", "fs", "-put", "-f", f.name, "/test-warehouse/" +
+                unique_database + "_bad_udf.ll"])
+      if vector.get_value('exec_option')['disable_codegen']:
+        self.run_test_case('QueryTest/udf-errors', vector, use_db=unique_database)
+    finally:
+      if os.path.exists(f.name):
+        os.remove(f.name)
+      call(["hadoop", "fs", "-rm", "-f", "/test-warehouse/" +
+                  unique_database + "_bad_udf.ll"])
+      os.close(fd)
 
   # Run serially because this will blow the process limit, potentially causing other
   # queries to fail


[07/15] impala git commit: IMPALA-4835: Part 1: simplify I/O mgr mem mgmt and cancellation

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/io/request-context.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-context.h b/be/src/runtime/io/request-context.h
index 9807805..737a16c 100644
--- a/be/src/runtime/io/request-context.h
+++ b/be/src/runtime/io/request-context.h
@@ -24,9 +24,7 @@
 namespace impala {
 namespace io {
 /// A request context is used to group together I/O requests belonging to a client of the
-/// I/O manager for management and scheduling. For most I/O manager clients it is an
-/// opaque pointer, but some clients may need to include this header, e.g. to make the
-/// unique_ptr<DiskIoRequestContext> destructor work correctly.
+/// I/O manager for management and scheduling.
 ///
 /// Implementation Details
 /// ======================
@@ -83,7 +81,51 @@ namespace io {
 /// behind at most one write range.
 class RequestContext {
  public:
-  ~RequestContext() { DCHECK_EQ(state_, Inactive) << "Must be unregistered."; }
+  ~RequestContext() {
+    DCHECK_EQ(state_, Inactive) << "Must be unregistered. " << DebugString();
+  }
+
+  /// Cancel the context asynchronously. All outstanding requests are cancelled
+  /// asynchronously. This does not need to be called if the context finishes normally.
+  /// Calling GetNext() on any scan ranges belonging to this RequestContext will return
+  /// CANCELLED (or another error, if an error was encountered for that scan range before
+  /// it is cancelled).
+  void Cancel();
+
+  bool IsCancelled() {
+    boost::unique_lock<boost::mutex> lock(lock_);
+    return state_ == Cancelled;
+  }
+
+  int64_t queue_size() const { return num_ready_buffers_.Load(); }
+  int64_t bytes_read_local() const { return bytes_read_local_.Load(); }
+  int64_t bytes_read_short_circuit() const { return bytes_read_short_circuit_.Load(); }
+  int64_t bytes_read_dn_cache() const { return bytes_read_dn_cache_.Load(); }
+  int num_remote_ranges() const { return num_remote_ranges_.Load(); }
+  int64_t unexpected_remote_bytes() const { return unexpected_remote_bytes_.Load(); }
+
+  int cached_file_handles_hit_count() const {
+    return cached_file_handles_hit_count_.Load();
+  }
+
+  int cached_file_handles_miss_count() const {
+    return cached_file_handles_miss_count_.Load();
+  }
+
+  void set_bytes_read_counter(RuntimeProfile::Counter* bytes_read_counter) {
+    bytes_read_counter_ = bytes_read_counter;
+  }
+
+  void set_read_timer(RuntimeProfile::Counter* read_timer) { read_timer_ = read_timer; }
+
+  void set_active_read_thread_counter(
+      RuntimeProfile::Counter* active_read_thread_counter) {
+   active_read_thread_counter_ = active_read_thread_counter;
+  }
+
+  void set_disks_accessed_bitmap(RuntimeProfile::Counter* disks_accessed_bitmap) {
+    disks_accessed_bitmap_ = disks_accessed_bitmap;
+  }
 
  private:
   DISALLOW_COPY_AND_ASSIGN(RequestContext);
@@ -108,11 +150,24 @@ class RequestContext {
 
   RequestContext(DiskIoMgr* parent, int num_disks, MemTracker* tracker);
 
+  /// Allocates a buffer to read into with size between
+  /// max('buffer_size', 'min_buffer_size_') and 'max_buffer_size_'.
+  /// Does not acquire 'lock_'.
+  /// TODO: allocate using the buffer pool client associated with this reader.
+  Status AllocBuffer(ScanRange* range, int64_t buffer_size,
+      std::unique_ptr<BufferDescriptor>* buffer);
+
+  /// Cleans up a buffer. If the buffer was allocated with AllocBuffer(), frees the buffer
+  /// memory and release the consumption to the client MemTracker. Otherwise (e.g. a
+  /// client or HDFS cache buffer), just prepares the descriptor to be destroyed.
+  /// After this is called, buffer->buffer() is NULL. Does not acquire 'lock_'.
+  void FreeBuffer(BufferDescriptor* buffer);
+
   /// Decrements the number of active disks for this reader.  If the disk count
   /// goes to 0, the disk complete condition variable is signaled.
-  /// Reader lock must be taken before this call.
-  void DecrementDiskRefCount() {
-    // boost doesn't let us dcheck that the reader lock is taken
+  /// 'lock_' must be held via 'lock'.
+  void DecrementDiskRefCount(const boost::unique_lock<boost::mutex>& lock) {
+    DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
     DCHECK_GT(num_disks_with_ranges_, 0);
     if (--num_disks_with_ranges_ == 0) {
       disks_complete_cond_var_.NotifyAll();
@@ -129,25 +184,25 @@ class RequestContext {
 
   /// Adds range to in_flight_ranges, scheduling this reader on the disk threads
   /// if necessary.
-  /// Reader lock must be taken before this.
-  void ScheduleScanRange(ScanRange* range) {
+  /// 'lock_' must be held via 'lock'
+  void ScheduleScanRange(const boost::unique_lock<boost::mutex>& lock, ScanRange* range) {
+    DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
     DCHECK_EQ(state_, Active);
     DCHECK(range != NULL);
     RequestContext::PerDiskState& state = disk_states_[range->disk_id()];
     state.in_flight_ranges()->Enqueue(range);
-    state.ScheduleContext(this, range->disk_id());
+    state.ScheduleContext(lock, this, range->disk_id());
   }
 
-  /// Cancels the context with status code 'status'
-  void Cancel(const Status& status);
-
   /// Cancel the context if not already cancelled, wait for all scan ranges to finish
   /// and mark the context as inactive, after which it cannot be used.
   void CancelAndMarkInactive();
 
   /// Adds request range to disk queue for this request context. Currently,
   /// schedule_immediately must be false is RequestRange is a write range.
-  void AddRequestRange(RequestRange* range, bool schedule_immediately);
+  /// Caller must hold 'lock_' via 'lock'.
+  void AddRequestRange(const boost::unique_lock<boost::mutex>& lock,
+      RequestRange* range, bool schedule_immediately);
 
   /// Validates invariants of reader.  Reader lock must be taken beforehand.
   bool Validate() const;
@@ -159,6 +214,7 @@ class RequestContext {
   DiskIoMgr* const parent_;
 
   /// Memory used for this reader.  This is unowned by this object.
+  /// TODO: replace with bp client
   MemTracker* const mem_tracker_;
 
   /// Total bytes read for this reader
@@ -187,7 +243,7 @@ class RequestContext {
   /// Total number of bytes from remote reads that were expected to be local.
   AtomicInt64 unexpected_remote_bytes_{0};
 
-  /// The number of buffers that have been returned to the reader (via GetNext) that the
+  /// The number of buffers that have been returned to the reader (via GetNext()) that the
   /// reader has not returned. Only included for debugging and diagnostics.
   AtomicInt32 num_buffers_in_reader_{0};
 
@@ -227,9 +283,6 @@ class RequestContext {
   /// Current state of the reader
   State state_ = Active;
 
-  /// Status of this reader.  Set to non-ok if cancelled.
-  Status status_;
-
   /// The number of disks with scan ranges remaining (always equal to the sum of
   /// disks with ranges).
   int num_disks_with_ranges_ = 0;
@@ -237,7 +290,7 @@ class RequestContext {
   /// This is the list of ranges that are expected to be cached on the DN.
   /// When the reader asks for a new range (GetNextScanRange()), we first
   /// return ranges from this list.
-  InternalQueue<ScanRange> cached_ranges_;
+  InternalList<ScanRange> cached_ranges_;
 
   /// A list of ranges that should be returned in subsequent calls to
   /// GetNextRange.
@@ -246,11 +299,11 @@ class RequestContext {
   /// Populating it preemptively means we make worse scheduling decisions.
   /// We currently populate one range per disk.
   /// TODO: think about this some more.
-  InternalQueue<ScanRange> ready_to_start_ranges_;
+  InternalList<ScanRange> ready_to_start_ranges_;
   ConditionVariable ready_to_start_ranges_cv_; // used with lock_
 
   /// Ranges that are blocked due to back pressure on outgoing buffers.
-  InternalQueue<ScanRange> blocked_ranges_;
+  InternalList<ScanRange> blocked_ranges_;
 
   /// Condition variable for UnregisterContext() to wait for all disks to complete
   ConditionVariable disks_complete_cond_var_;
@@ -270,21 +323,9 @@ class RequestContext {
       next_scan_range_to_start_ = range;
     }
 
-    /// We need to have a memory barrier to prevent this load from being reordered
-    /// with num_threads_in_op(), since these variables are set without the reader
-    /// lock taken
-    bool is_on_queue() const {
-      bool b = is_on_queue_;
-      __sync_synchronize();
-      return b;
-    }
+    bool is_on_queue() const { return is_on_queue_.Load() != 0; }
 
-    int num_threads_in_op() const {
-      int v = num_threads_in_op_.Load();
-      // TODO: determine whether this barrier is necessary for any callsites.
-      AtomicUtil::MemoryBarrier();
-      return v;
-    }
+    int num_threads_in_op() const { return num_threads_in_op_.Load(); }
 
     const InternalQueue<ScanRange>* unstarted_scan_ranges() const {
       return &unstarted_scan_ranges_;
@@ -303,26 +344,41 @@ class RequestContext {
     InternalQueue<RequestRange>* in_flight_ranges() { return &in_flight_ranges_; }
 
     /// Schedules the request context on this disk if it's not already on the queue.
-    /// Context lock must be taken before this.
-    void ScheduleContext(RequestContext* context, int disk_id);
-
-    /// Increment the ref count on reader.  We need to track the number of threads per
-    /// reader per disk that are in the unlocked hdfs read code section. This is updated
-    /// by multiple threads without a lock so we need to use an atomic int.
-    void IncrementRequestThreadAndDequeue() {
+    /// context->lock_ must be held by the caller via 'context_lock'.
+    void ScheduleContext(const boost::unique_lock<boost::mutex>& context_lock,
+        RequestContext* context, int disk_id);
+
+    /// Increment the count of disk threads that have a reference to this context. These
+    /// threads do not hold any locks while reading from HDFS, so we need to prevent the
+    /// RequestContext from being destroyed underneath them.
+    ///
+    /// The caller does not need to hold 'lock_', so this can execute concurrently with
+    /// itself and DecrementDiskThread().
+    void IncrementDiskThreadAndDequeue() {
+      /// Incrementing 'num_threads_in_op_' first so that there is no window when other
+      /// threads see 'is_on_queue_ == num_threads_in_op_ == 0' and think there are no
+      /// references left to this context.
       num_threads_in_op_.Add(1);
-      is_on_queue_ = false;
+      is_on_queue_.Store(0);
     }
 
-    void DecrementRequestThread() { num_threads_in_op_.Add(-1); }
-
-    /// Decrement request thread count and do final cleanup if this is the last
-    /// thread. RequestContext lock must be taken before this.
-    void DecrementRequestThreadAndCheckDone(RequestContext* context) {
-      num_threads_in_op_.Add(-1); // Also acts as a barrier.
-      if (!is_on_queue_ && num_threads_in_op_.Load() == 0 && !done_) {
-        // This thread is the last one for this reader on this disk, do final cleanup
-        context->DecrementDiskRefCount();
+    /// Decrement the count of disks threads with a reference to this context. Does final
+    /// cleanup if the context is cancelled and this is the last thread for the disk.
+    /// context->lock_ must be held by the caller via 'context_lock'.
+    void DecrementDiskThread(const boost::unique_lock<boost::mutex>& context_lock,
+        RequestContext* context) {
+      DCHECK(context_lock.mutex() == &context->lock_ && context_lock.owns_lock());
+      num_threads_in_op_.Add(-1);
+
+      if (context->state_ != Cancelled) {
+        DCHECK_EQ(context->state_, Active);
+        return;
+      }
+      // The state is cancelled, check to see if we're the last thread to touch the
+      // context on this disk. We need to load 'is_on_queue_' and 'num_threads_in_op_'
+      // in this order to avoid a race with IncrementDiskThreadAndDequeue().
+      if (is_on_queue_.Load() == 0 && num_threads_in_op_.Load() == 0 && !done_) {
+        context->DecrementDiskRefCount(context_lock);
         done_ = true;
       }
     }
@@ -335,7 +391,12 @@ class RequestContext {
     bool done_ = true;
 
     /// For each disk, keeps track if the context is on this disk's queue, indicating
-    /// the disk must do some work for this context. The disk needs to do work in 4 cases:
+    /// the disk must do some work for this context. 1 means that the context is on the
+    /// disk queue, 0 means that it's not on the queue (either because it has on ranges
+    /// active for the disk or because a disk thread dequeued the context and is
+    /// currently processing a request).
+    ///
+    /// The disk needs to do work in 4 cases:
     ///  1) in_flight_ranges is not empty, the disk needs to read for this reader.
     ///  2) next_range_to_start is NULL, the disk needs to prepare a scan range to be
     ///     read next.
@@ -346,7 +407,15 @@ class RequestContext {
     /// useful that can be done. If there's nothing useful, the disk queue will wake up
     /// and then remove the reader from the queue. Doing this causes thrashing of the
     /// threads.
-    bool is_on_queue_ = false;
+    ///
+    /// This variable is important during context cancellation because it indicates
+    /// whether a queue has a reference to the context that must be released before
+    /// the context is considered unregistered. Atomically set to false after
+    /// incrementing 'num_threads_in_op_' when dequeueing so that there is no window
+    /// when other threads see 'is_on_queue_ == num_threads_in_op_ == 0' and think there
+    /// are no references left to this context.
+    /// TODO: this could be combined with 'num_threads_in_op_' to be a single refcount.
+    AtomicInt32 is_on_queue_{0};
 
     /// For each disks, the number of request ranges that have not been fully read.
     /// In the non-cancellation path, this will hit 0, and done will be set to true
@@ -376,11 +445,11 @@ class RequestContext {
     /// range to ready_to_start_ranges_.
     ScanRange* next_scan_range_to_start_ = nullptr;
 
-    /// For each disk, the number of threads issuing the underlying read/write on behalf
-    /// of this context. There are a few places where we release the context lock, do some
-    /// work, and then grab the lock again.  Because we don't hold the lock for the
-    /// entire operation, we need this ref count to keep track of which thread should do
-    /// final resource cleanup during cancellation.
+    /// For each disk, the number of disk threads issuing the underlying read/write on
+    /// behalf of this context. There are a few places where we release the context lock,
+    /// do some work, and then grab the lock again.  Because we don't hold the lock for
+    /// the entire operation, we need this ref count to keep track of which thread should
+    /// do final resource cleanup during cancellation.
     /// Only the thread that sees the count at 0 should do the final cleanup.
     AtomicInt32 num_threads_in_op_{0};
 

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/io/request-ranges.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-ranges.h b/be/src/runtime/io/request-ranges.h
index 222f847..ab0810a 100644
--- a/be/src/runtime/io/request-ranges.h
+++ b/be/src/runtime/io/request-ranges.h
@@ -23,6 +23,7 @@
 
 #include <boost/thread/mutex.hpp>
 
+#include "common/atomic.h"
 #include "common/hdfs.h"
 #include "common/status.h"
 #include "util/condition-variable.h"
@@ -55,14 +56,6 @@ class BufferDescriptor {
   /// Returns the offset within the scan range that this buffer starts at
   int64_t scan_range_offset() const { return scan_range_offset_; }
 
-  /// Transfer ownership of buffer memory from 'mem_tracker_' to 'dst' and set
-  /// 'mem_tracker_' to 'dst'. 'mem_tracker_' and 'dst' must be non-NULL. Does not
-  /// check memory limits on 'dst': the caller should check the memory limit if a
-  /// different memory limit may apply to 'dst'. If the buffer was a client-provided
-  /// buffer, transferring is not allowed.
-  /// TODO: IMPALA-3209: revisit this as part of scanner memory usage revamp.
-  void TransferOwnership(MemTracker* dst);
-
  private:
   friend class DiskIoMgr;
   friend class ScanRange;
@@ -71,8 +64,7 @@ class BufferDescriptor {
   /// Create a buffer descriptor for a new reader, range and data buffer. The buffer
   /// memory should already be accounted against 'mem_tracker'.
   BufferDescriptor(DiskIoMgr* io_mgr, RequestContext* reader,
-      ScanRange* scan_range, uint8_t* buffer, int64_t buffer_len,
-      MemTracker* mem_tracker);
+      ScanRange* scan_range, uint8_t* buffer, int64_t buffer_len);
 
   /// Return true if this is a cached buffer owned by HDFS.
   bool is_cached() const;
@@ -86,14 +78,11 @@ class BufferDescriptor {
   /// Reader that this buffer is for.
   RequestContext* const reader_;
 
-  /// The current tracker this buffer is associated with. After initialisation,
-  /// NULL for cached buffers and non-NULL for all other buffers.
-  MemTracker* mem_tracker_;
-
   /// Scan range that this buffer is for. Non-NULL when initialised.
   ScanRange* const scan_range_;
 
-  /// buffer with the read contents
+  /// Buffer for the read contents. Must be set to NULL in RequestContext::FreeBuffer()
+  /// before destruction of the descriptor.
   uint8_t* buffer_;
 
   /// length of buffer_. For buffers from cached reads, the length is 0.
@@ -105,9 +94,6 @@ class BufferDescriptor {
   /// true if the current scan range is complete
   bool eosr_ = false;
 
-  /// Status of the read to this buffer. if status is not ok, 'buffer' is nullptr
-  Status status_;
-
   int64_t scan_range_offset_ = 0;
 };
 
@@ -236,10 +222,17 @@ class ScanRange : public RequestRange {
   /// Only one thread can be in GetNext() at any time.
   Status GetNext(std::unique_ptr<BufferDescriptor>* buffer) WARN_UNUSED_RESULT;
 
-  /// Cancel this scan range. This cleans up all queued buffers and
-  /// wakes up any threads blocked on GetNext().
-  /// Status is the reason the range was cancelled. Must not be ok().
-  /// Status is returned to the user in GetNext().
+  /// Returns the buffer to the scan range. This must be called for every buffer
+  /// returned by GetNext(). After calling this, the buffer descriptor is invalid
+  /// and cannot be accessed.
+  void ReturnBuffer(std::unique_ptr<BufferDescriptor> buffer);
+
+  /// Cancel this scan range. This cleans up all queued buffers and wakes up any threads
+  /// blocked on GetNext(). Status is a non-ok status with the reason the range was
+  /// cancelled, e.g. CANCELLED if the range was cancelled because it was not needed, or
+  /// another error if an error was encountered while scanning the range. Status is
+  /// returned to the any callers of GetNext(). If a thread is currently blocked in
+  /// GetNext(), it is woken up.
   void Cancel(const Status& status);
 
   /// return a descriptive string for debug.
@@ -263,10 +256,6 @@ class ScanRange : public RequestRange {
   bool EnqueueBuffer(const boost::unique_lock<boost::mutex>& reader_lock,
       std::unique_ptr<BufferDescriptor> buffer);
 
-  /// Cleanup any queued buffers (i.e. due to cancellation). This cannot
-  /// be called with any locks taken.
-  void CleanupQueuedBuffers();
-
   /// Validates the internal state of this range. lock_ must be taken
   /// before calling this.
   bool Validate();
@@ -283,6 +272,10 @@ class ScanRange : public RequestRange {
   /// exclusive use by this scan range. The scan range is the exclusive owner of the
   /// file handle, and the file handle is destroyed in Close().
   /// All local OS files are opened using normal OS file APIs.
+  ///
+  /// If an error is encountered during opening, returns a status describing the error.
+  /// If the scan range was cancelled, returns the reason for cancellation. Otherwise, on
+  /// success, returns OK.
   Status Open(bool use_file_handle_cache) WARN_UNUSED_RESULT;
 
   /// Closes the file for this range. This function only modifies state in this range.
@@ -290,6 +283,10 @@ class ScanRange : public RequestRange {
 
   /// Reads from this range into 'buffer', which has length 'buffer_len' bytes. Returns
   /// the number of bytes read. The read position in this scan range is updated.
+  ///
+  /// If an error is encountered during reading, returns a status describing the error.
+  /// If the scan range was cancelled, returns the reason for cancellation. Otherwise, on
+  /// success, returns OK.
   Status Read(uint8_t* buffer, int64_t buffer_len, int64_t* bytes_read,
       bool* eosr) WARN_UNUSED_RESULT;
 
@@ -307,6 +304,23 @@ class ScanRange : public RequestRange {
   Status ReadFromCache(const boost::unique_lock<boost::mutex>& reader_lock,
       bool* read_succeeded) WARN_UNUSED_RESULT;
 
+  /// Cleans up a buffer that was not returned to the client.
+  /// Either ReturnBuffer() or CleanUpBuffer() is called for every BufferDescriptor.
+  /// This function will acquire 'lock_' and may acquire 'hdfs_lock_'.
+  void CleanUpBuffer(std::unique_ptr<BufferDescriptor> buffer);
+
+  /// Same as CleanUpBuffer() except the caller must already hold 'lock_' via
+  /// 'scan_range_lock'.
+  void CleanUpBufferLocked(const boost::unique_lock<boost::mutex>& scan_range_lock,
+      std::unique_ptr<BufferDescriptor> buffer);
+
+  /// Returns true if no more buffers will be returned to clients in the future,
+  /// either because of hitting eosr or cancellation.
+  bool all_buffers_returned(const boost::unique_lock<boost::mutex>& lock) const {
+    DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
+    return !cancel_status_.ok() || (eosr_queued_ && ready_buffers_.empty());
+  }
+
   /// Pointer to caller specified metadata. This is untouched by the io manager
   /// and the caller can put whatever auxiliary data in here.
   void* meta_data_ = nullptr;
@@ -323,6 +337,9 @@ class ScanRange : public RequestRange {
   /// TODO: we can do more with this
   bool expected_local_ = false;
 
+  /// Last modified time of the file associated with the scan range. Set in Reset().
+  int64_t mtime_;
+
   /// Total number of bytes read remotely. This is necessary to maintain a count of
   /// the number of remote scan ranges. Since IO statistics can be collected multiple
   /// times for a scan range, it is necessary to keep some state about whether this
@@ -378,26 +395,29 @@ class ScanRange : public RequestRange {
   /// Number of bytes read so far for this scan range
   int bytes_read_;
 
-  /// Status for this range. This is non-ok if is_cancelled_ is true.
-  /// Note: an individual range can fail without the RequestContext being
-  /// cancelled. This allows us to skip individual ranges.
-  Status status_;
+  /// The number of buffers that have been returned to a client via GetNext() that have
+  /// not yet been returned with ReturnBuffer().
+  int num_buffers_in_reader_ = 0;
 
   /// If true, the last buffer for this scan range has been queued.
+  /// If this is true and 'ready_buffers_' is empty, then no more buffers will be
+  /// returned to the caller by this scan range.
   bool eosr_queued_ = false;
 
-  /// If true, the last buffer for this scan range has been returned.
-  bool eosr_returned_ = false;
-
   /// If true, this scan range has been removed from the reader's in_flight_ranges
   /// queue because the ready_buffers_ queue is full.
   bool blocked_on_queue_ = false;
 
-  /// IO buffers that are queued for this scan range.
-  /// Condition variable for GetNext
-  ConditionVariable buffer_ready_cv_;
+  /// IO buffers that are queued for this scan range. When Cancel() is called
+  /// this is drained by the cancelling thread. I.e. this is always empty if
+  /// 'cancel_status_' is not OK.
   std::deque<std::unique_ptr<BufferDescriptor>> ready_buffers_;
 
+  /// Condition variable for threads in GetNext() that are waiting for the next buffer.
+  /// Signalled when a buffer is enqueued in 'ready_buffers_' or the scan range is
+  /// cancelled.
+  ConditionVariable buffer_ready_cv_;
+
   /// Lock that should be taken during hdfs calls. Only one thread (the disk reading
   /// thread) calls into hdfs at a time so this lock does not have performance impact.
   /// This lock only serves to coordinate cleanup. Specifically it serves to ensure
@@ -406,11 +426,16 @@ class ScanRange : public RequestRange {
   /// If this lock and lock_ need to be taken, lock_ must be taken first.
   boost::mutex hdfs_lock_;
 
-  /// If true, this scan range has been cancelled.
-  bool is_cancelled_ = false;
-
-  /// Last modified time of the file associated with the scan range
-  int64_t mtime_;
+  /// If non-OK, this scan range has been cancelled. This status is the reason for
+  /// cancellation - CANCELLED if cancelled without error, or another status if an
+  /// error caused cancellation. Note that a range can be cancelled without cancelling
+  /// the owning context. This means that ranges can be cancelled or hit errors without
+  /// aborting all scan ranges.
+  //
+  /// Writers must hold both 'lock_' and 'hdfs_lock_'. Readers must hold either 'lock_'
+  /// or 'hdfs_lock_'. This prevents the range from being cancelled while any thread
+  /// is inside a critical section.
+  Status cancel_status_;
 };
 
 /// Used to specify data to be written to a file and offset.

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/io/scan-range.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/scan-range.cc b/be/src/runtime/io/scan-range.cc
index 21daa96..1ffba00 100644
--- a/be/src/runtime/io/scan-range.cc
+++ b/be/src/runtime/io/scan-range.cc
@@ -45,19 +45,15 @@ DEFINE_int64(adls_read_chunk_size, 128 * 1024, "The maximum read chunk size to u
 bool ScanRange::EnqueueBuffer(
     const unique_lock<mutex>& reader_lock, unique_ptr<BufferDescriptor> buffer) {
   DCHECK(reader_lock.mutex() == &reader_->lock_ && reader_lock.owns_lock());
+  DCHECK(buffer->buffer_ != nullptr) << "Cannot enqueue freed buffer";
   {
     unique_lock<mutex> scan_range_lock(lock_);
     DCHECK(Validate()) << DebugString();
-    DCHECK(!eosr_returned_);
     DCHECK(!eosr_queued_);
-    if (is_cancelled_) {
-      // Return the buffer, this range has been cancelled
-      if (buffer->buffer_ != nullptr) {
-        io_mgr_->num_buffers_in_readers_.Add(1);
-        reader_->num_buffers_in_reader_.Add(1);
-      }
+    if (!cancel_status_.ok()) {
+      // This range has been cancelled, no need to enqueue the buffer.
       reader_->num_used_buffers_.Add(-1);
-      io_mgr_->ReturnBuffer(move(buffer));
+      CleanUpBufferLocked(scan_range_lock, move(buffer));
       return false;
     }
     reader_->num_ready_buffers_.Add(1);
@@ -67,9 +63,7 @@ bool ScanRange::EnqueueBuffer(
     DCHECK_LE(ready_buffers_.size(), DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT);
     blocked_on_queue_ = ready_buffers_.size() == DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT;
   }
-
   buffer_ready_cv_.NotifyOne();
-
   return blocked_on_queue_;
 }
 
@@ -78,56 +72,48 @@ Status ScanRange::GetNext(unique_ptr<BufferDescriptor>* buffer) {
   bool eosr;
   {
     unique_lock<mutex> scan_range_lock(lock_);
-    if (eosr_returned_) return Status::OK();
     DCHECK(Validate()) << DebugString();
+    // No more buffers to return - return the cancel status or OK if not cancelled.
+    if (all_buffers_returned(scan_range_lock)) return cancel_status_;
 
-    while (ready_buffers_.empty() && !is_cancelled_) {
+    while (ready_buffers_.empty() && cancel_status_.ok()) {
       buffer_ready_cv_.Wait(scan_range_lock);
     }
-
-    if (is_cancelled_) {
-      DCHECK(!status_.ok());
-      return status_;
-    }
+    /// Propagate cancellation to the client if it happened while we were waiting.
+    RETURN_IF_ERROR(cancel_status_);
 
     // Remove the first ready buffer from the queue and return it
     DCHECK(!ready_buffers_.empty());
     DCHECK_LE(ready_buffers_.size(), DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT);
     *buffer = move(ready_buffers_.front());
     ready_buffers_.pop_front();
-    eosr_returned_ = (*buffer)->eosr();
     eosr = (*buffer)->eosr();
   }
 
   // Update tracking counters. The buffer has now moved from the IoMgr to the
   // caller.
-  io_mgr_->num_buffers_in_readers_.Add(1);
-  reader_->num_buffers_in_reader_.Add(1);
   reader_->num_ready_buffers_.Add(-1);
   reader_->num_used_buffers_.Add(-1);
   if (eosr) reader_->num_finished_ranges_.Add(1);
 
-  Status status = (*buffer)->status_;
-  if (!status.ok()) {
-    io_mgr_->ReturnBuffer(move(*buffer));
-    return status;
-  }
-
   unique_lock<mutex> reader_lock(reader_->lock_);
 
   DCHECK(reader_->Validate()) << endl << reader_->DebugString();
   if (reader_->state_ == RequestContext::Cancelled) {
     reader_->blocked_ranges_.Remove(this);
-    Cancel(reader_->status_);
-    io_mgr_->ReturnBuffer(move(*buffer));
-    return status_;
+    Cancel(Status::CANCELLED);
+    CleanUpBuffer(move(*buffer));
+    return Status::CANCELLED;
   }
 
+  // At this point success is guaranteed so increment counters for returned buffers.
+  reader_->num_buffers_in_reader_.Add(1);
   {
     // Check to see if we can re-schedule a blocked range. Note that EnqueueBuffer()
     // may have been called after we released 'lock_' above so we need to re-check
     // whether the queue is full.
     unique_lock<mutex> scan_range_lock(lock_);
+    ++num_buffers_in_reader_;
     if (blocked_on_queue_
         && ready_buffers_.size() < DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT
         && !eosr_queued_) {
@@ -135,51 +121,81 @@ Status ScanRange::GetNext(unique_ptr<BufferDescriptor>* buffer) {
       // This scan range was blocked and is no longer, add it to the reader
       // queue again.
       reader_->blocked_ranges_.Remove(this);
-      reader_->ScheduleScanRange(this);
+      reader_->ScheduleScanRange(reader_lock, this);
     }
   }
   return Status::OK();
 }
 
+void ScanRange::ReturnBuffer(unique_ptr<BufferDescriptor> buffer_desc) {
+  reader_->num_buffers_in_reader_.Add(-1);
+  {
+    unique_lock<mutex> scan_range_lock(lock_);
+    --num_buffers_in_reader_;
+    CleanUpBufferLocked(scan_range_lock, move(buffer_desc));
+  }
+}
+
+void ScanRange::CleanUpBuffer(unique_ptr<BufferDescriptor> buffer_desc) {
+  unique_lock<mutex> scan_range_lock(lock_);
+  CleanUpBufferLocked(scan_range_lock, move(buffer_desc));
+}
+
+void ScanRange::CleanUpBufferLocked(
+    const boost::unique_lock<boost::mutex>& scan_range_lock,
+    unique_ptr<BufferDescriptor> buffer_desc) {
+  DCHECK(scan_range_lock.mutex() == &lock_ && scan_range_lock.owns_lock());
+  DCHECK(buffer_desc != nullptr);
+  DCHECK_EQ(this, buffer_desc->scan_range_);
+  buffer_desc->reader_->FreeBuffer(buffer_desc.get());
+
+  if (all_buffers_returned(scan_range_lock) && num_buffers_in_reader_ == 0) {
+    // Close the scan range if there are no more buffers in the reader and no more buffers
+    // will be returned to readers in future. Close() is idempotent so it is ok to call
+    // multiple times during cleanup so long as the range is actually finished.
+    Close();
+  }
+}
+
 void ScanRange::Cancel(const Status& status) {
   // Cancelling a range that was never started, ignore.
   if (io_mgr_ == nullptr) return;
 
   DCHECK(!status.ok());
   {
-    // Grab both locks to make sure that all working threads see is_cancelled_.
+    // Grab both locks to make sure that we don't change 'cancel_status_' while other
+    // threads are in critical sections.
     unique_lock<mutex> scan_range_lock(lock_);
-    unique_lock<mutex> hdfs_lock(hdfs_lock_);
-    DCHECK(Validate()) << DebugString();
-    if (is_cancelled_) return;
-    is_cancelled_ = true;
-    status_ = status;
+    {
+      unique_lock<mutex> hdfs_lock(hdfs_lock_);
+      DCHECK(Validate()) << DebugString();
+      // If already cancelled, preserve the original reason for cancellation. The first
+      // thread to set 'cancel_status_' does the cleanup below.
+      RETURN_VOID_IF_ERROR(cancel_status_);
+      cancel_status_ = status;
+    }
+
+    /// Clean up 'ready_buffers_' while still holding 'lock_' to prevent other threads
+    /// from seeing inconsistent state.
+    reader_->num_used_buffers_.Add(-ready_buffers_.size());
+    reader_->num_ready_buffers_.Add(-ready_buffers_.size());
+    while (!ready_buffers_.empty()) {
+      CleanUpBufferLocked(scan_range_lock, move(ready_buffers_.front()));
+      ready_buffers_.pop_front();
+    }
   }
   buffer_ready_cv_.NotifyAll();
-  CleanupQueuedBuffers();
 
   // For cached buffers, we can't close the range until the cached buffer is returned.
-  // Close() is called from DiskIoMgr::ReturnBuffer().
+  // Close() is called from ScanRange::CleanUpBufferLocked().
   if (external_buffer_tag_ != ExternalBufferTag::CACHED_BUFFER) Close();
 }
 
-void ScanRange::CleanupQueuedBuffers() {
-  DCHECK(is_cancelled_);
-  io_mgr_->num_buffers_in_readers_.Add(ready_buffers_.size());
-  reader_->num_buffers_in_reader_.Add(ready_buffers_.size());
-  reader_->num_used_buffers_.Add(-ready_buffers_.size());
-  reader_->num_ready_buffers_.Add(-ready_buffers_.size());
-
-  while (!ready_buffers_.empty()) {
-    io_mgr_->ReturnBuffer(move(ready_buffers_.front()));
-    ready_buffers_.pop_front();
-  }
-}
-
 string ScanRange::DebugString() const {
   stringstream ss;
   ss << "file=" << file_ << " disk_id=" << disk_id_ << " offset=" << offset_
      << " len=" << len_ << " bytes_read=" << bytes_read_
+     << " cancel_status=" << cancel_status_.GetDetail()
      << " buffer_queue=" << ready_buffers_.size()
      << " hdfs_file=" << exclusive_hdfs_fh_;
   return ss.str();
@@ -187,14 +203,12 @@ string ScanRange::DebugString() const {
 
 bool ScanRange::Validate() {
   if (bytes_read_ > len_) {
-    LOG(WARNING) << "Bytes read tracking is wrong. Shouldn't read past the scan range."
+    LOG(ERROR) << "Bytes read tracking is wrong. Shouldn't read past the scan range."
                  << " bytes_read_=" << bytes_read_ << " len_=" << len_;
     return false;
   }
-  if (eosr_returned_ && !eosr_queued_) {
-    LOG(WARNING) << "Returned eosr to reader before finishing reading the scan range"
-                 << " eosr_returned_=" << eosr_returned_
-                 << " eosr_queued_=" << eosr_queued_;
+  if (!cancel_status_.ok() && !ready_buffers_.empty()) {
+    LOG(ERROR) << "Cancelled range should not have queued buffers " << DebugString();
     return false;
   }
   return true;
@@ -203,13 +217,14 @@ bool ScanRange::Validate() {
 ScanRange::ScanRange()
   : RequestRange(RequestType::READ),
     num_remote_bytes_(0),
-    external_buffer_tag_(ExternalBufferTag::NO_BUFFER),
-    mtime_(-1) {}
+    external_buffer_tag_(ExternalBufferTag::NO_BUFFER) {}
 
 ScanRange::~ScanRange() {
   DCHECK(exclusive_hdfs_fh_ == nullptr) << "File was not closed.";
   DCHECK(external_buffer_tag_ != ExternalBufferTag::CACHED_BUFFER)
       << "Cached buffer was not released.";
+  DCHECK_EQ(0, ready_buffers_.size());
+  DCHECK_EQ(0, num_buffers_in_reader_);
 }
 
 void ScanRange::Reset(hdfsFS fs, const char* file, int64_t len, int64_t offset,
@@ -253,16 +268,15 @@ void ScanRange::InitInternal(DiskIoMgr* io_mgr, RequestContext* reader) {
   local_file_ = nullptr;
   exclusive_hdfs_fh_ = nullptr;
   bytes_read_ = 0;
-  is_cancelled_ = false;
+  cancel_status_ = Status::OK();
   eosr_queued_= false;
-  eosr_returned_= false;
   blocked_on_queue_ = false;
   DCHECK(Validate()) << DebugString();
 }
 
 Status ScanRange::Open(bool use_file_handle_cache) {
   unique_lock<mutex> hdfs_lock(hdfs_lock_);
-  if (is_cancelled_) return Status::CANCELLED;
+  RETURN_IF_ERROR(cancel_status_);
 
   if (fs_ != nullptr) {
     if (exclusive_hdfs_fh_ != nullptr) return Status::OK();
@@ -386,7 +400,7 @@ int64_t ScanRange::MaxReadChunkSize() const {
 Status ScanRange::Read(
     uint8_t* buffer, int64_t buffer_len, int64_t* bytes_read, bool* eosr) {
   unique_lock<mutex> hdfs_lock(hdfs_lock_);
-  if (is_cancelled_) return Status::CANCELLED;
+  RETURN_IF_ERROR(cancel_status_);
 
   *eosr = false;
   *bytes_read = 0;
@@ -523,7 +537,7 @@ Status ScanRange::ReadFromCache(
 
   {
     unique_lock<mutex> hdfs_lock(hdfs_lock_);
-    if (is_cancelled_) return Status::CANCELLED;
+    RETURN_IF_ERROR(cancel_status_);
 
     DCHECK(exclusive_hdfs_fh_ != nullptr);
     DCHECK(external_buffer_tag_ == ExternalBufferTag::NO_BUFFER);
@@ -561,7 +575,7 @@ Status ScanRange::ReadFromCache(
   // 'mem_tracker' is nullptr because the memory is owned by the HDFS java client,
   // not the Impala backend.
   unique_ptr<BufferDescriptor> desc = unique_ptr<BufferDescriptor>(new BufferDescriptor(
-      io_mgr_, reader_, this, reinterpret_cast<uint8_t*>(buffer), 0, nullptr));
+      io_mgr_, reader_, this, reinterpret_cast<uint8_t*>(buffer), 0));
   desc->len_ = bytes_read;
   desc->scan_range_offset_ = 0;
   desc->eosr_ = true;

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/mem-tracker.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/mem-tracker.h b/be/src/runtime/mem-tracker.h
index 10a3424..8d28f8f 100644
--- a/be/src/runtime/mem-tracker.h
+++ b/be/src/runtime/mem-tracker.h
@@ -107,7 +107,6 @@ class MemTracker {
   /// destruction to prevent other threads from getting a reference to the MemTracker
   /// via its parent. Only used to deregister the query-level MemTracker from the
   /// global hierarchy.
-  /// TODO: IMPALA-3200: this is also used by BufferedBlockMgr, which will be deleted.
   void CloseAndUnregisterFromParent();
 
   /// Include counters from a ReservationTracker in logs and other diagnostics.

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/test-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/test-env.cc b/be/src/runtime/test-env.cc
index 770eaba..0d6c6f0 100644
--- a/be/src/runtime/test-env.cc
+++ b/be/src/runtime/test-env.cc
@@ -49,7 +49,7 @@ Status TestEnv::Init() {
   exec_env_.reset(new ExecEnv);
   // Populate the ExecEnv state that the backend tests need.
   exec_env_->mem_tracker_.reset(new MemTracker(-1, "Process"));
-  RETURN_IF_ERROR(exec_env_->disk_io_mgr()->Init(exec_env_->process_mem_tracker()));
+  RETURN_IF_ERROR(exec_env_->disk_io_mgr()->Init());
   exec_env_->metrics_.reset(new MetricGroup("test-env-metrics"));
   exec_env_->tmp_file_mgr_.reset(new TmpFileMgr);
   if (have_tmp_file_mgr_args_) {

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/tmp-file-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr-test.cc b/be/src/runtime/tmp-file-mgr-test.cc
index 3091c58..9161b63 100644
--- a/be/src/runtime/tmp-file-mgr-test.cc
+++ b/be/src/runtime/tmp-file-mgr-test.cc
@@ -25,6 +25,7 @@
 #include <gtest/gtest.h>
 
 #include "common/init.h"
+#include "runtime/io/request-context.h"
 #include "runtime/test-env.h"
 #include "runtime/tmp-file-mgr-internal.h"
 #include "runtime/tmp-file-mgr.h"
@@ -134,7 +135,7 @@ class TmpFileMgrTest : public ::testing::Test {
 
   /// Helper to cancel the FileGroup RequestContext.
   static void CancelIoContext(TmpFileMgr::FileGroup* group) {
-    group->io_mgr_->CancelContext(group->io_ctx_.get());
+    group->io_ctx_->Cancel();
   }
 
   /// Helper to get the # of bytes allocated by the group. Validates that the sum across

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/tmp-file-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr.cc b/be/src/runtime/tmp-file-mgr.cc
index 3807670..7b00179 100644
--- a/be/src/runtime/tmp-file-mgr.cc
+++ b/be/src/runtime/tmp-file-mgr.cc
@@ -426,7 +426,7 @@ Status TmpFileMgr::FileGroup::WaitForAsyncRead(WriteHandle* handle, MemRange buf
   }
 exit:
   // Always return the buffer before exiting to avoid leaking it.
-  if (io_mgr_buffer != nullptr) io_mgr_->ReturnBuffer(move(io_mgr_buffer));
+  if (io_mgr_buffer != nullptr) handle->read_range_->ReturnBuffer(move(io_mgr_buffer));
   handle->read_range_ = nullptr;
   return status;
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/util/impalad-metrics.cc
----------------------------------------------------------------------
diff --git a/be/src/util/impalad-metrics.cc b/be/src/util/impalad-metrics.cc
index 8f5f1be..32320d8 100644
--- a/be/src/util/impalad-metrics.cc
+++ b/be/src/util/impalad-metrics.cc
@@ -46,12 +46,6 @@ const char* ImpaladMetricKeys::HASH_TABLE_TOTAL_BYTES =
     "impala-server.hash-table.total-bytes";
 const char* ImpaladMetricKeys::IO_MGR_NUM_OPEN_FILES =
     "impala-server.io-mgr.num-open-files";
-const char* ImpaladMetricKeys::IO_MGR_NUM_BUFFERS =
-    "impala-server.io-mgr.num-buffers";
-const char* ImpaladMetricKeys::IO_MGR_TOTAL_BYTES =
-    "impala-server.io-mgr.total-bytes";
-const char* ImpaladMetricKeys::IO_MGR_NUM_UNUSED_BUFFERS =
-    "impala-server.io-mgr.num-unused-buffers";
 const char* ImpaladMetricKeys::IO_MGR_BYTES_READ =
     "impala-server.io-mgr.bytes-read";
 const char* ImpaladMetricKeys::IO_MGR_LOCAL_BYTES_READ =
@@ -208,11 +202,8 @@ void ImpaladMetrics::CreateMetrics(MetricGroup* m) {
       ImpaladMetricKeys::NUM_FILES_OPEN_FOR_INSERT, 0);
 
   // Initialize IO mgr metrics
-  IO_MGR_NUM_OPEN_FILES = m->AddGauge(ImpaladMetricKeys::IO_MGR_NUM_OPEN_FILES, 0);
-  IO_MGR_NUM_BUFFERS = m->AddGauge(ImpaladMetricKeys::IO_MGR_NUM_BUFFERS, 0);
-  IO_MGR_TOTAL_BYTES = m->AddGauge(ImpaladMetricKeys::IO_MGR_TOTAL_BYTES, 0);
-  IO_MGR_NUM_UNUSED_BUFFERS = m->AddGauge(
-      ImpaladMetricKeys::IO_MGR_NUM_UNUSED_BUFFERS, 0);
+  IO_MGR_NUM_OPEN_FILES = m->AddGauge(
+      ImpaladMetricKeys::IO_MGR_NUM_OPEN_FILES, 0);
   IO_MGR_NUM_CACHED_FILE_HANDLES = m->AddGauge(
       ImpaladMetricKeys::IO_MGR_NUM_CACHED_FILE_HANDLES, 0);
   IO_MGR_NUM_FILE_HANDLES_OUTSTANDING = m->AddGauge(

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/util/impalad-metrics.h
----------------------------------------------------------------------
diff --git a/be/src/util/impalad-metrics.h b/be/src/util/impalad-metrics.h
index f32e3fa..a62c4c6 100644
--- a/be/src/util/impalad-metrics.h
+++ b/be/src/util/impalad-metrics.h
@@ -67,15 +67,6 @@ class ImpaladMetricKeys {
   /// Number of files currently opened by the io mgr
   static const char* IO_MGR_NUM_OPEN_FILES;
 
-  /// Number of IO buffers allocated by the io mgr
-  static const char* IO_MGR_NUM_BUFFERS;
-
-  /// Number of bytes used by IO buffers (used and unused).
-  static const char* IO_MGR_TOTAL_BYTES;
-
-  /// Number of IO buffers that are currently unused (and can be GC'ed)
-  static const char* IO_MGR_NUM_UNUSED_BUFFERS;
-
   /// Total number of bytes read by the io mgr
   static const char* IO_MGR_BYTES_READ;
 


[14/15] impala git commit: IMPALA-4835: Part 2: Allocate scan range buffers upfront

Posted by ta...@apache.org.
IMPALA-4835: Part 2: Allocate scan range buffers upfront

This change is a step towards reserving memory for buffers from the
buffer pool and constraining per-scanner memory requirements. This
change restructures the DiskIoMgr code so that each ScanRange operates
with a fixed set of buffers that are allocated upfront and recycled as
the I/O mgr works through the ScanRange.

One major change is that ScanRanges get blocked when a buffer is not
available and get unblocked when a client returns a buffer via
ReturnBuffer(). I was able to remove the logic to maintain the
blocked_ranges_ list by instead adding a separate set with all ranges
that are active.

There is also some miscellaneous cleanup included - e.g. reducing the
amount of code devoted to maintaining counters and metrics.

One tricky part of the existing code was the it called
IssueInitialRanges() with empty lists of files and depended on
DiskIoMgr::AddScanRanges() to not check for cancellation in that case.
See  IMPALA-6564. I changed the logic to not try to issue ranges for
empty lists of files.

I plan to merge this along with the actual buffer pool switch, but
separated it out to allow review of the DiskIoMgr changes separate from
other aspects of the buffer pool switchover.

Testing:
* Ran core and exhaustive tests.

Change-Id: Ia243bf8b62feeea602b122e0503ea4ba7d3ee70f
Reviewed-on: http://gerrit.cloudera.org:8080/8707
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Tim Armstrong <ta...@cloudera.com>
Reviewed-on: http://gerrit.cloudera.org:8080/9416


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/0b6fab73
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/0b6fab73
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/0b6fab73

Branch: refs/heads/2.x
Commit: 0b6fab7315c6894d4f96d52fb022305be63884a2
Parents: 3b3bf87
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Nov 29 11:30:51 2017 -0800
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Fri Feb 23 22:51:01 2018 +0000

----------------------------------------------------------------------
 be/src/exec/base-sequence-scanner.cc         |   1 +
 be/src/exec/base-sequence-scanner.h          |   3 +-
 be/src/exec/hdfs-lzo-text-scanner.cc         |   1 +
 be/src/exec/hdfs-parquet-scanner.cc          |  25 +-
 be/src/exec/hdfs-parquet-scanner.h           |   6 +-
 be/src/exec/hdfs-scan-node-base.cc           |  34 ++-
 be/src/exec/hdfs-scan-node-mt.cc             |  13 +-
 be/src/exec/hdfs-scan-node.cc                |  32 ++-
 be/src/exec/scanner-context.cc               |  15 +-
 be/src/runtime/bufferpool/buffer-pool.h      |   1 +
 be/src/runtime/io/disk-io-mgr-internal.h     |  16 ++
 be/src/runtime/io/disk-io-mgr-stress-test.cc |  13 +-
 be/src/runtime/io/disk-io-mgr-stress.cc      |  12 +-
 be/src/runtime/io/disk-io-mgr-test.cc        | 232 +++++++++++++++--
 be/src/runtime/io/disk-io-mgr.cc             | 295 +++++++++++-----------
 be/src/runtime/io/disk-io-mgr.h              | 257 ++++++++++---------
 be/src/runtime/io/request-context.cc         | 152 +++++------
 be/src/runtime/io/request-context.h          | 153 ++++++-----
 be/src/runtime/io/request-ranges.h           |  94 +++++--
 be/src/runtime/io/scan-range.cc              | 205 ++++++++++-----
 be/src/runtime/tmp-file-mgr.cc               |  14 +-
 be/src/runtime/tmp-file-mgr.h                |  17 +-
 be/src/util/bit-util-test.cc                 |  11 +
 be/src/util/bit-util.h                       |   8 +-
 24 files changed, 1020 insertions(+), 590 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/base-sequence-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/base-sequence-scanner.cc b/be/src/exec/base-sequence-scanner.cc
index 9cb6330..9d95b0b 100644
--- a/be/src/exec/base-sequence-scanner.cc
+++ b/be/src/exec/base-sequence-scanner.cc
@@ -46,6 +46,7 @@ static const int MIN_SYNC_READ_SIZE = 64 * 1024; // bytes
 
 Status BaseSequenceScanner::IssueInitialRanges(HdfsScanNodeBase* scan_node,
     const vector<HdfsFileDesc*>& files) {
+  DCHECK(!files.empty());
   // Issue just the header range for each file.  When the header is complete,
   // we'll issue the splits for that file.  Splits cannot be processed until the
   // header is parsed (the header object is then shared across splits for that file).

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/base-sequence-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/base-sequence-scanner.h b/be/src/exec/base-sequence-scanner.h
index 887ff6f..3c2326e 100644
--- a/be/src/exec/base-sequence-scanner.h
+++ b/be/src/exec/base-sequence-scanner.h
@@ -47,7 +47,8 @@ class ScannerContext;
 /// situation, causing the block to be incorrectly skipped.
 class BaseSequenceScanner : public HdfsScanner {
  public:
-  /// Issue the initial ranges for all sequence container files.
+  /// Issue the initial ranges for all sequence container files. 'files' must not be
+  /// empty.
   static Status IssueInitialRanges(HdfsScanNodeBase* scan_node,
                                    const std::vector<HdfsFileDesc*>& files)
                                    WARN_UNUSED_RESULT;

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/hdfs-lzo-text-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-lzo-text-scanner.cc b/be/src/exec/hdfs-lzo-text-scanner.cc
index 88ae295..8af89f2 100644
--- a/be/src/exec/hdfs-lzo-text-scanner.cc
+++ b/be/src/exec/hdfs-lzo-text-scanner.cc
@@ -62,6 +62,7 @@ HdfsScanner* HdfsLzoTextScanner::GetHdfsLzoTextScanner(
 
 Status HdfsLzoTextScanner::IssueInitialRanges(HdfsScanNodeBase* scan_node,
     const vector<HdfsFileDesc*>& files) {
+  DCHECK(!files.empty());
   if (LzoIssueInitialRanges == NULL) {
     lock_guard<SpinLock> l(lzo_load_lock_);
     if (library_load_status_.ok()) {

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index bb3d091..0188f08 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -69,6 +69,7 @@ const string PARQUET_MEM_LIMIT_EXCEEDED =
 
 Status HdfsParquetScanner::IssueInitialRanges(HdfsScanNodeBase* scan_node,
     const vector<HdfsFileDesc*>& files) {
+  DCHECK(!files.empty());
   vector<ScanRange*> footer_ranges;
   for (int i = 0; i < files.size(); ++i) {
     // If the file size is less than 12 bytes, it is an invalid Parquet file.
@@ -1439,8 +1440,10 @@ Status HdfsParquetScanner::ProcessFooter() {
         BufferOpts::ReadInto(metadata_buffer.buffer(), metadata_size));
 
     unique_ptr<BufferDescriptor> io_buffer;
-    RETURN_IF_ERROR(
-        io_mgr->AddScanRange(scan_node_->reader_context(), metadata_range, true));
+    bool needs_buffers;
+    RETURN_IF_ERROR(io_mgr->StartScanRange(
+          scan_node_->reader_context(), metadata_range, &needs_buffers));
+    DCHECK(!needs_buffers) << "Already provided a buffer";
     RETURN_IF_ERROR(metadata_range->GetNext(&io_buffer));
     DCHECK_EQ(io_buffer->buffer(), metadata_buffer.buffer());
     DCHECK_EQ(io_buffer->len(), metadata_size);
@@ -1735,12 +1738,18 @@ Status HdfsParquetScanner::InitScalarColumns(
   }
   DCHECK_EQ(col_ranges.size(), num_scalar_readers);
 
-  // Issue all the column chunks to the io mgr and have them scheduled immediately.
-  // This means these ranges aren't returned via DiskIoMgr::GetNextRange and
-  // instead are scheduled to be read immediately.
-  RETURN_IF_ERROR(scan_node_->runtime_state()->io_mgr()->AddScanRanges(
-      scan_node_->reader_context(), col_ranges, true));
-
+  DiskIoMgr* io_mgr = scan_node_->runtime_state()->io_mgr();
+  // Issue all the column chunks to the IoMgr. We scan through all columns at the same
+  // time so need to read from all of them concurrently.
+  for (ScanRange* col_range : col_ranges) {
+    bool needs_buffers;
+    RETURN_IF_ERROR(io_mgr->StartScanRange(
+        scan_node_->reader_context(), col_range, &needs_buffers));
+    if (needs_buffers) {
+      RETURN_IF_ERROR(io_mgr->AllocateBuffersForRange(
+          scan_node_->reader_context(), col_range, 3 * io_mgr->max_buffer_size()));
+    }
+  }
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/hdfs-parquet-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.h b/be/src/exec/hdfs-parquet-scanner.h
index f0043b5..1fc3239 100644
--- a/be/src/exec/hdfs-parquet-scanner.h
+++ b/be/src/exec/hdfs-parquet-scanner.h
@@ -69,8 +69,8 @@ class BoolColumnReader;
 /// the split size, the mid point guarantees that we have at least 50% of the row group in
 /// the current split. ProcessSplit() then computes the column ranges for these row groups
 /// and submits them to the IoMgr for immediate scheduling (so they don't surface in
-/// DiskIoMgr::GetNextRange()). Scheduling them immediately also guarantees they are all
-/// read at once.
+/// DiskIoMgr::GetNextUnstartedRange()). Scheduling them immediately also guarantees they
+/// are all read at once.
 ///
 /// Like the other scanners, each parquet scanner object is one to one with a
 /// ScannerContext. Unlike the other scanners though, the context will have multiple
@@ -328,7 +328,7 @@ class HdfsParquetScanner : public HdfsScanner {
   virtual ~HdfsParquetScanner() {}
 
   /// Issue just the footer range for each file.  We'll then parse the footer and pick
-  /// out the columns we want.
+  /// out the columns we want. 'files' must not be empty.
   static Status IssueInitialRanges(HdfsScanNodeBase* scan_node,
                                    const std::vector<HdfsFileDesc*>& files)
                                    WARN_UNUSED_RESULT;

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/hdfs-scan-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index 0660b9b..861d5dc 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -449,18 +449,27 @@ Status HdfsScanNodeBase::IssueInitialScanRanges(RuntimeState* state) {
     }
   }
 
-  // Issue initial ranges for all file types.
-  RETURN_IF_ERROR(HdfsParquetScanner::IssueInitialRanges(this,
-      matching_per_type_files[THdfsFileFormat::PARQUET]));
-  RETURN_IF_ERROR(HdfsTextScanner::IssueInitialRanges(this,
-      matching_per_type_files[THdfsFileFormat::TEXT]));
-  RETURN_IF_ERROR(BaseSequenceScanner::IssueInitialRanges(this,
-      matching_per_type_files[THdfsFileFormat::SEQUENCE_FILE]));
-  RETURN_IF_ERROR(BaseSequenceScanner::IssueInitialRanges(this,
-      matching_per_type_files[THdfsFileFormat::RC_FILE]));
-  RETURN_IF_ERROR(BaseSequenceScanner::IssueInitialRanges(this,
-      matching_per_type_files[THdfsFileFormat::AVRO]));
-
+  // Issue initial ranges for all file types. Only call functions for file types that
+  // actually exist - trying to add empty lists of ranges can result in spurious
+  // CANCELLED errors - see IMPALA-6564.
+  for (const auto& entry : matching_per_type_files) {
+    if (entry.second.empty()) continue;
+    switch (entry.first) {
+      case THdfsFileFormat::PARQUET:
+        RETURN_IF_ERROR(HdfsParquetScanner::IssueInitialRanges(this, entry.second));
+        break;
+      case THdfsFileFormat::TEXT:
+        RETURN_IF_ERROR(HdfsTextScanner::IssueInitialRanges(this, entry.second));
+        break;
+      case THdfsFileFormat::SEQUENCE_FILE:
+      case THdfsFileFormat::RC_FILE:
+      case THdfsFileFormat::AVRO:
+        RETURN_IF_ERROR(BaseSequenceScanner::IssueInitialRanges(this, entry.second));
+        break;
+      default:
+        DCHECK(false) << "Unexpected file type " << entry.first;
+    }
+  }
   return Status::OK();
 }
 
@@ -519,6 +528,7 @@ ScanRange* HdfsScanNodeBase::AllocateScanRange(hdfsFS fs, const char* file,
 
 Status HdfsScanNodeBase::AddDiskIoRanges(
     const vector<ScanRange*>& ranges, int num_files_queued) {
+  DCHECK(!progress_.done()) << "Don't call AddScanRanges() after all ranges finished.";
   RETURN_IF_ERROR(runtime_state_->io_mgr()->AddScanRanges(reader_context_.get(), ranges));
   num_unqueued_files_.Add(-num_files_queued);
   DCHECK_GE(num_unqueued_files_.Load(), 0);

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/hdfs-scan-node-mt.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-mt.cc b/be/src/exec/hdfs-scan-node-mt.cc
index 7ea4d80..be75677 100644
--- a/be/src/exec/hdfs-scan-node-mt.cc
+++ b/be/src/exec/hdfs-scan-node-mt.cc
@@ -26,6 +26,7 @@
 
 #include "gen-cpp/PlanNodes_types.h"
 
+using namespace impala::io;
 using std::stringstream;
 
 namespace impala {
@@ -76,13 +77,19 @@ Status HdfsScanNodeMt::GetNext(RuntimeState* state, RowBatch* row_batch, bool* e
       scanner_->Close(row_batch);
       scanner_.reset();
     }
-    RETURN_IF_ERROR(
-        runtime_state_->io_mgr()->GetNextRange(reader_context_.get(), &scan_range_));
-    if (scan_range_ == NULL) {
+    DiskIoMgr* io_mgr = runtime_state_->io_mgr();
+    bool needs_buffers;
+    RETURN_IF_ERROR(io_mgr->GetNextUnstartedRange(
+        reader_context_.get(), &scan_range_, &needs_buffers));
+    if (scan_range_ == nullptr) {
       *eos = true;
       StopAndFinalizeCounters();
       return Status::OK();
     }
+    if (needs_buffers) {
+      RETURN_IF_ERROR(io_mgr->AllocateBuffersForRange(reader_context_.get(), scan_range_,
+          3 * io_mgr->max_buffer_size()));
+    }
     ScanRangeMetadata* metadata =
         static_cast<ScanRangeMetadata*>(scan_range_->meta_data());
     int64_t partition_id = metadata->partition_id;

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/hdfs-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node.cc b/be/src/exec/hdfs-scan-node.cc
index b32a743..f9d71e9 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -179,10 +179,10 @@ Status HdfsScanNode::Prepare(RuntimeState* state) {
   if (per_type_files_[THdfsFileFormat::PARQUET].size() > 0) {
     // Parquet files require buffers per column
     scanner_thread_bytes_required_ =
-        materialized_slots_.size() * 3 * runtime_state_->io_mgr()->max_read_buffer_size();
+        materialized_slots_.size() * 3 * runtime_state_->io_mgr()->max_buffer_size();
   } else {
     scanner_thread_bytes_required_ =
-        3 * runtime_state_->io_mgr()->max_read_buffer_size();
+        3 * runtime_state_->io_mgr()->max_buffer_size();
   }
   // scanner_thread_bytes_required_ now contains the IoBuffer requirement.
   // Next we add in the other memory the scanner thread will use.
@@ -376,6 +376,7 @@ void HdfsScanNode::ThreadTokenAvailableCb(ThreadResourceMgr::ResourcePool* pool)
 void HdfsScanNode::ScannerThread() {
   SCOPED_THREAD_COUNTER_MEASUREMENT(scanner_thread_counters());
   SCOPED_THREAD_COUNTER_MEASUREMENT(runtime_state_->total_thread_statistics());
+  DiskIoMgr* io_mgr = runtime_state_->io_mgr();
 
   // Make thread-local copy of filter contexts to prune scan ranges, and to pass to the
   // scanner for finer-grained filtering. Use a thread-local MemPool for the filter
@@ -422,21 +423,28 @@ void HdfsScanNode::ScannerThread() {
     // to return if there's an error.
     ranges_issued_barrier_.Wait(SCANNER_THREAD_WAIT_TIME_MS, &unused);
 
-    ScanRange* scan_range;
-    // Take a snapshot of num_unqueued_files_ before calling GetNextRange().
+    // Take a snapshot of num_unqueued_files_ before calling GetNextUnstartedRange().
     // We don't want num_unqueued_files_ to go to zero between the return from
-    // GetNextRange() and the check for when all ranges are complete.
+    // GetNextUnstartedRange() and the check for when all ranges are complete.
     int num_unqueued_files = num_unqueued_files_.Load();
     // TODO: the Load() acts as an acquire barrier.  Is this needed? (i.e. any earlier
     // stores that need to complete?)
     AtomicUtil::MemoryBarrier();
+    ScanRange* scan_range;
+    bool needs_buffers;
     Status status =
-        runtime_state_->io_mgr()->GetNextRange(reader_context_.get(), &scan_range);
+        io_mgr->GetNextUnstartedRange(reader_context_.get(), &scan_range, &needs_buffers);
 
-    if (status.ok() && scan_range != NULL) {
-      // Got a scan range. Process the range end to end (in this thread).
-      status = ProcessSplit(filter_status.ok() ? filter_ctxs : vector<FilterContext>(),
-          &expr_results_pool, scan_range);
+    if (status.ok() && scan_range != nullptr) {
+      if (needs_buffers) {
+        status = io_mgr->AllocateBuffersForRange(
+            reader_context_.get(), scan_range, 3 * io_mgr->max_buffer_size());
+      }
+      if (status.ok()) {
+        // Got a scan range. Process the range end to end (in this thread).
+        status = ProcessSplit(filter_status.ok() ? filter_ctxs : vector<FilterContext>(),
+            &expr_results_pool, scan_range);
+      }
     }
 
     if (!status.ok()) {
@@ -466,8 +474,8 @@ void HdfsScanNode::ScannerThread() {
       // TODO: Based on the usage pattern of all_ranges_started_, it looks like it is not
       // needed to acquire the lock in x86.
       unique_lock<mutex> l(lock_);
-      // All ranges have been queued and GetNextRange() returned NULL. This means that
-      // every range is either done or being processed by another thread.
+      // All ranges have been queued and GetNextUnstartedRange() returned NULL. This means
+      // that every range is either done or being processed by another thread.
       all_ranges_started_ = true;
       break;
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/exec/scanner-context.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/scanner-context.cc b/be/src/exec/scanner-context.cc
index 79e7a85..0abf82f 100644
--- a/be/src/exec/scanner-context.cc
+++ b/be/src/exec/scanner-context.cc
@@ -101,6 +101,7 @@ void ScannerContext::Stream::ReleaseCompletedResources(bool done) {
 
 Status ScannerContext::Stream::GetNextBuffer(int64_t read_past_size) {
   DCHECK_EQ(0, io_buffer_bytes_left_);
+  DiskIoMgr* io_mgr = parent_->state_->io_mgr();
   if (UNLIKELY(parent_->cancelled())) return Status::CANCELLED;
   if (io_buffer_ != nullptr) ReturnIoBuffer();
 
@@ -121,7 +122,7 @@ Status ScannerContext::Stream::GetNextBuffer(int64_t read_past_size) {
     SCOPED_TIMER(parent_->state_->total_storage_wait_timer());
 
     int64_t read_past_buffer_size = 0;
-    int64_t max_buffer_size = parent_->state_->io_mgr()->max_read_buffer_size();
+    int64_t max_buffer_size = io_mgr->max_buffer_size();
     if (!read_past_size_cb_.empty()) read_past_buffer_size = read_past_size_cb_(offset);
     if (read_past_buffer_size <= 0) {
       // Either no callback was set or the callback did not return an estimate. Use
@@ -143,8 +144,16 @@ Status ScannerContext::Stream::GetNextBuffer(int64_t read_past_size) {
     ScanRange* range = parent_->scan_node_->AllocateScanRange(
         scan_range_->fs(), filename(), read_past_buffer_size, offset, partition_id,
         scan_range_->disk_id(), false, BufferOpts::Uncached());
-    RETURN_IF_ERROR(parent_->state_->io_mgr()->AddScanRange(
-        parent_->scan_node_->reader_context(), range, true));
+    bool needs_buffers;
+    RETURN_IF_ERROR(io_mgr->StartScanRange(
+        parent_->scan_node_->reader_context(), range, &needs_buffers));
+    if (needs_buffers) {
+      // Allocate fresh buffers. The buffers for 'scan_range_' should be released now
+      // since we hit EOS.
+      RETURN_IF_ERROR(io_mgr->AllocateBuffersForRange(
+          parent_->scan_node_->reader_context(), range,
+          3 * io_mgr->max_buffer_size()));
+    }
     RETURN_IF_ERROR(range->GetNext(&io_buffer_));
     DCHECK(io_buffer_->eosr());
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/bufferpool/buffer-pool.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/buffer-pool.h b/be/src/runtime/bufferpool/buffer-pool.h
index 285aacb..d14da63 100644
--- a/be/src/runtime/bufferpool/buffer-pool.h
+++ b/be/src/runtime/bufferpool/buffer-pool.h
@@ -37,6 +37,7 @@
 
 namespace impala {
 
+class MemTracker;
 class ReservationTracker;
 class RuntimeProfile;
 class SystemAllocator;

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/disk-io-mgr-internal.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-internal.h b/be/src/runtime/io/disk-io-mgr-internal.h
index 3fc3895..2d32487 100644
--- a/be/src/runtime/io/disk-io-mgr-internal.h
+++ b/be/src/runtime/io/disk-io-mgr-internal.h
@@ -35,9 +35,25 @@
 #include "util/filesystem-util.h"
 #include "util/hdfs-util.h"
 #include "util/impalad-metrics.h"
+#include "util/runtime-profile-counters.h"
 
 /// This file contains internal structures shared between submodules of the IoMgr. Users
 /// of the IoMgr do not need to include this file.
+
+// Macros to work around counters sometimes not being provided.
+// TODO: fix things so that counters are always non-NULL.
+#define COUNTER_ADD_IF_NOT_NULL(c, v) \
+  do { \
+    ::impala::RuntimeProfile::Counter* __ctr__ = (c); \
+    if (__ctr__ != nullptr) __ctr__->Add(v); \
+ } while (false);
+
+#define COUNTER_BITOR_IF_NOT_NULL(c, v) \
+  do { \
+    ::impala::RuntimeProfile::Counter* __ctr__ = (c); \
+    if (__ctr__ != nullptr) __ctr__->BitOr(v); \
+ } while (false);
+
 namespace impala {
 namespace io {
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/disk-io-mgr-stress-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-stress-test.cc b/be/src/runtime/io/disk-io-mgr-stress-test.cc
index 45b36ed..0e41a6f 100644
--- a/be/src/runtime/io/disk-io-mgr-stress-test.cc
+++ b/be/src/runtime/io/disk-io-mgr-stress-test.cc
@@ -16,7 +16,10 @@
 // under the License.
 
 #include "runtime/io/disk-io-mgr-stress.h"
-#include "util/cpu-info.h"
+
+#include "common/init.h"
+#include "runtime/test-env.h"
+#include "service/fe-support.h"
 #include "util/string-parser.h"
 
 #include "common/names.h"
@@ -35,10 +38,10 @@ const int NUM_CLIENTS = 10;
 const bool TEST_CANCELLATION = true;
 
 int main(int argc, char** argv) {
-  google::InitGoogleLogging(argv[0]);
-  CpuInfo::Init();
-  OsInfo::Init();
-  impala::InitThreading();
+  InitCommonRuntime(argc, argv, true, TestInfo::BE_TEST);
+  InitFeSupport();
+  TestEnv test_env;
+  ABORT_IF_ERROR(test_env.Init());
   int duration_sec = DEFAULT_DURATION_SEC;
 
   if (argc == 2) {

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/disk-io-mgr-stress.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-stress.cc b/be/src/runtime/io/disk-io-mgr-stress.cc
index cfe71ab..ba1ad92 100644
--- a/be/src/runtime/io/disk-io-mgr-stress.cc
+++ b/be/src/runtime/io/disk-io-mgr-stress.cc
@@ -38,6 +38,9 @@ static const int MAX_FILE_LEN = 1024;
 static const int MIN_READ_BUFFER_SIZE = 64;
 static const int MAX_READ_BUFFER_SIZE = 128;
 
+// Maximum bytes to allocate per scan range.
+static const int MAX_BUFFER_BYTES_PER_SCAN_RANGE = MAX_READ_BUFFER_SIZE * 3;
+
 static const int CANCEL_READER_PERIOD_MS = 20;  // in ms
 
 static void CreateTempFile(const char* filename, const char* data) {
@@ -110,9 +113,16 @@ void DiskIoMgrStress::ClientThread(int client_id) {
 
     while (!eos) {
       ScanRange* range;
-      Status status = io_mgr_->GetNextRange(client->reader.get(), &range);
+      bool needs_buffers;
+      Status status =
+          io_mgr_->GetNextUnstartedRange(client->reader.get(), &range, &needs_buffers);
       CHECK(status.ok() || status.IsCancelled());
       if (range == NULL) break;
+      if (needs_buffers) {
+        status = io_mgr_->AllocateBuffersForRange(
+            client->reader.get(), range, MAX_BUFFER_BYTES_PER_SCAN_RANGE);
+        CHECK(status.ok()) << status.GetDetail();
+      }
 
       while (true) {
         unique_ptr<BufferDescriptor> buffer;

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/disk-io-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-test.cc b/be/src/runtime/io/disk-io-mgr-test.cc
index e099285..95ea184 100644
--- a/be/src/runtime/io/disk-io-mgr-test.cc
+++ b/be/src/runtime/io/disk-io-mgr-test.cc
@@ -41,7 +41,7 @@ DECLARE_int32(num_remote_hdfs_io_threads);
 DECLARE_int32(num_s3_io_threads);
 DECLARE_int32(num_adls_io_threads);
 
-const int MIN_BUFFER_SIZE = 512;
+const int MIN_BUFFER_SIZE = 128;
 const int MAX_BUFFER_SIZE = 1024;
 const int LARGE_MEM_LIMIT = 1024 * 1024 * 1024;
 
@@ -122,7 +122,12 @@ class DiskIoMgrTest : public testing::Test {
   static void ValidateSyncRead(DiskIoMgr* io_mgr, RequestContext* reader,
       ScanRange* range, const char* expected, int expected_len = -1) {
     unique_ptr<BufferDescriptor> buffer;
-    ASSERT_OK(io_mgr->AddScanRange(reader, range, true));
+    bool needs_buffers;
+    ASSERT_OK(io_mgr->StartScanRange(reader, range, &needs_buffers));
+    if (needs_buffers) {
+      ASSERT_OK(io_mgr->AllocateBuffersForRange(
+          reader, range, io_mgr->max_buffer_size()));
+    }
     ASSERT_OK(range->GetNext(&buffer));
     ASSERT_TRUE(buffer != nullptr);
     EXPECT_EQ(buffer->len(), range->len());
@@ -161,9 +166,14 @@ class DiskIoMgrTest : public testing::Test {
     int num_ranges = 0;
     while (max_ranges == 0 || num_ranges < max_ranges) {
       ScanRange* range;
-      Status status = io_mgr->GetNextRange(reader, &range);
+      bool needs_buffers;
+      Status status = io_mgr->GetNextUnstartedRange(reader, &range, &needs_buffers);
       ASSERT_TRUE(status.ok() || status.code() == expected_status.code());
       if (range == nullptr) break;
+      if (needs_buffers) {
+        ASSERT_OK(io_mgr->AllocateBuffersForRange(
+            reader, range, io_mgr->max_buffer_size() * 3));
+      }
       ValidateScanRange(io_mgr, range, expected_result, expected_len, expected_status);
       num_ranges_processed->Add(1);
       ++num_ranges;
@@ -509,7 +519,6 @@ TEST_F(DiskIoMgrTest, SyncReadTest) {
       // Issue some reads before the async ones are issued
       ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
       ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
-
       vector<ScanRange*> ranges;
       for (int i = 0; i < len; ++i) {
         int disk_id = i % num_disks;
@@ -641,15 +650,23 @@ TEST_F(DiskIoMgrTest, MemLimits) {
 
     bool hit_mem_limit_exceeded = false;
     char result[strlen(data) + 1];
-    // Keep reading new ranges without returning buffers. This forces us
-    // to go over the limit eventually.
+    // Keep starting new ranges without returning buffers. This forces us to go over
+    // the limit eventually.
     while (true) {
       memset(result, 0, strlen(data) + 1);
       ScanRange* range = nullptr;
-      Status status = io_mgr.GetNextRange(reader.get(), &range);
+      bool needs_buffers;
+      Status status = io_mgr.GetNextUnstartedRange(reader.get(), &range, &needs_buffers);
       ASSERT_TRUE(status.ok() || status.IsMemLimitExceeded());
       hit_mem_limit_exceeded |= status.IsMemLimitExceeded();
       if (range == nullptr) break;
+      DCHECK(needs_buffers);
+      status = io_mgr.AllocateBuffersForRange(reader.get(), range, MAX_BUFFER_SIZE * 3);
+      ASSERT_TRUE(status.ok() || status.IsMemLimitExceeded());
+      if (status.IsMemLimitExceeded()) {
+        hit_mem_limit_exceeded = true;
+        continue;
+      }
 
       while (true) {
         unique_ptr<BufferDescriptor> buffer;
@@ -925,34 +942,140 @@ TEST_F(DiskIoMgrTest, PartialRead) {
   const char* data = "the quick brown fox jumped over the lazy dog";
   int len = strlen(data);
   int read_len = len + 1000; // Read past end of file.
+  // Test with various buffer sizes to exercise different code paths, e.g.
+  // * the truncated data ends exactly on a buffer boundary
+  // * the data is split between many buffers
+  // * the data fits in one buffer
+  const int64_t MIN_BUFFER_SIZE = 2;
+  vector<int64_t> max_buffer_sizes{4, 16, 32, 128, 1024, 4096};
   CreateTempFile(tmp_file, data);
 
   // Get mtime for file
   struct stat stat_val;
   stat(tmp_file, &stat_val);
 
-  scoped_ptr<DiskIoMgr> io_mgr(new DiskIoMgr(1, 1, 1, read_len, read_len));
+  for (int64_t max_buffer_size : max_buffer_sizes) {
+    DiskIoMgr io_mgr(1, 1, 1, MIN_BUFFER_SIZE, max_buffer_size);
 
-  ASSERT_OK(io_mgr->Init());
+    ASSERT_OK(io_mgr.Init());
+    MemTracker reader_mem_tracker;
+    unique_ptr<RequestContext> reader;
+    reader = io_mgr.RegisterContext(&reader_mem_tracker);
+
+    // We should not read past the end of file.
+    ScanRange* range = InitRange(tmp_file, 0, read_len, 0, stat_val.st_mtime);
+    unique_ptr<BufferDescriptor> buffer;
+    bool needs_buffers;
+    ASSERT_OK(io_mgr.StartScanRange(reader.get(), range, &needs_buffers));
+    if (needs_buffers) {
+      ASSERT_OK(io_mgr.AllocateBuffersForRange(reader.get(), range, 3 * max_buffer_size));
+    }
+
+    int64_t bytes_read = 0;
+    bool eosr = false;
+    do {
+      ASSERT_OK(range->GetNext(&buffer));
+      ASSERT_GE(buffer->buffer_len(), MIN_BUFFER_SIZE);
+      ASSERT_LE(buffer->buffer_len(), max_buffer_size);
+      ASSERT_LE(buffer->len(), len - bytes_read);
+      ASSERT_TRUE(memcmp(buffer->buffer(), data + bytes_read, buffer->len()) == 0);
+      bytes_read += buffer->len();
+      eosr = buffer->eosr();
+      // Should see eosr if we've read past the end of the file. If the data is an exact
+      // multiple of the max buffer size then we may read to the end of the file without
+      // noticing that it is eosr. Eosr will be returned on the next read in that case.
+      ASSERT_TRUE(bytes_read < len || buffer->eosr()
+          || (buffer->len() == max_buffer_size && len % max_buffer_size == 0))
+          << "max_buffer_size " << max_buffer_size << " bytes_read " << bytes_read
+          << "len " << len << " buffer->len() " << buffer->len()
+          << " buffer->buffer_len() " << buffer->buffer_len();
+      ASSERT_TRUE(buffer->len() > 0 || buffer->eosr());
+      range->ReturnBuffer(move(buffer));
+    } while (!eosr);
+
+    io_mgr.UnregisterContext(reader.get());
+    EXPECT_EQ(reader_mem_tracker.consumption(), 0);
+    EXPECT_EQ(mem_tracker.consumption(), 0);
+    pool_.Clear();
+  }
+}
+
+// Test zero-length scan range.
+TEST_F(DiskIoMgrTest, ZeroLengthScanRange) {
+  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
+  const char* data = "the quick brown fox jumped over the lazy dog";
+  const int64_t MIN_BUFFER_SIZE = 2;
+  const int64_t MAX_BUFFER_SIZE = 1024;
+  CreateTempFile(tmp_file, data);
+
+  // Get mtime for file
+  struct stat stat_val;
+  stat(tmp_file, &stat_val);
+
+  DiskIoMgr io_mgr(1, 1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
+
+  ASSERT_OK(io_mgr.Init());
   MemTracker reader_mem_tracker;
   unique_ptr<RequestContext> reader;
-  reader = io_mgr->RegisterContext(&reader_mem_tracker);
+  reader = io_mgr.RegisterContext(&reader_mem_tracker);
 
   // We should not read past the end of file.
-  ScanRange* range = InitRange(tmp_file, 0, read_len, 0, stat_val.st_mtime);
-  unique_ptr<BufferDescriptor> buffer;
-  ASSERT_OK(io_mgr->AddScanRange(reader.get(), range, true));
-  ASSERT_OK(range->GetNext(&buffer));
-  ASSERT_TRUE(buffer->eosr());
-  ASSERT_EQ(len, buffer->len());
-  ASSERT_TRUE(memcmp(buffer->buffer(), data, len) == 0);
-  range->ReturnBuffer(move(buffer));
+  ScanRange* range = InitRange(tmp_file, 0, 0, 0, stat_val.st_mtime);
+  bool needs_buffers;
+  Status status = io_mgr.StartScanRange(reader.get(), range, &needs_buffers);
+  ASSERT_EQ(TErrorCode::DISK_IO_ERROR, status.code());
 
-  io_mgr->UnregisterContext(reader.get());
-  pool_.Clear();
-  io_mgr.reset();
-  EXPECT_EQ(reader_mem_tracker.consumption(), 0);
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  status = io_mgr.AddScanRanges(reader.get(), vector<ScanRange*>({range}));
+  ASSERT_EQ(TErrorCode::DISK_IO_ERROR, status.code());
+
+  io_mgr.UnregisterContext(reader.get());
+}
+
+// Test what happens if don't call AllocateBuffersForRange() after trying to start a
+// range.
+TEST_F(DiskIoMgrTest, SkipAllocateBuffers) {
+  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
+  const char* data = "the quick brown fox jumped over the lazy dog";
+  int len = strlen(data);
+  const int64_t MIN_BUFFER_SIZE = 2;
+  const int64_t MAX_BUFFER_SIZE = 1024;
+  CreateTempFile(tmp_file, data);
+
+  // Get mtime for file
+  struct stat stat_val;
+  stat(tmp_file, &stat_val);
+
+  DiskIoMgr io_mgr(1, 1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
+
+  ASSERT_OK(io_mgr.Init());
+  MemTracker reader_mem_tracker;
+  unique_ptr<RequestContext> reader;
+  reader = io_mgr.RegisterContext(&reader_mem_tracker);
+
+  // We should not read past the end of file.
+  vector<ScanRange*> ranges;
+  for (int i = 0; i < 4; ++i) {
+    ranges.push_back(InitRange(tmp_file, 0, len, 0, stat_val.st_mtime));
+  }
+  bool needs_buffers;
+  // Test StartScanRange().
+  ASSERT_OK(io_mgr.StartScanRange(reader.get(), ranges[0], &needs_buffers));
+  EXPECT_TRUE(needs_buffers);
+  ASSERT_OK(io_mgr.StartScanRange(reader.get(), ranges[1], &needs_buffers));
+  EXPECT_TRUE(needs_buffers);
+
+  // Test AddScanRanges()/GetNextUnstartedRange().
+  ASSERT_OK(
+      io_mgr.AddScanRanges(reader.get(), vector<ScanRange*>({ranges[2], ranges[3]})));
+
+  // Cancel two directly, cancel the other two indirectly via the context.
+  ranges[0]->Cancel(Status::CANCELLED);
+  ranges[2]->Cancel(Status::CANCELLED);
+  reader->Cancel();
+
+  io_mgr.UnregisterContext(reader.get());
 }
 
 // Test reading into a client-allocated buffer.
@@ -978,7 +1101,9 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBuffer) {
     ScanRange* range = AllocateRange();
     range->Reset(nullptr, tmp_file, scan_len, 0, 0, true,
         BufferOpts::ReadInto(client_buffer.data(), buffer_len));
-    ASSERT_OK(io_mgr->AddScanRange(reader.get(), range, true));
+    bool needs_buffers;
+    ASSERT_OK(io_mgr->StartScanRange(reader.get(), range, &needs_buffers));
+    ASSERT_FALSE(needs_buffers);
 
     unique_ptr<BufferDescriptor> io_buffer;
     ASSERT_OK(range->GetNext(&io_buffer));
@@ -1016,7 +1141,9 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBufferError) {
     ScanRange* range = AllocateRange();
     range->Reset(nullptr, tmp_file, SCAN_LEN, 0, 0, true,
         BufferOpts::ReadInto(client_buffer.data(), SCAN_LEN));
-    ASSERT_OK(io_mgr->AddScanRange(reader.get(), range, true));
+    bool needs_buffers;
+    ASSERT_OK(io_mgr->StartScanRange(reader.get(), range, &needs_buffers));
+    ASSERT_FALSE(needs_buffers);
 
     /// Also test the cancellation path. Run multiple iterations since it is racy whether
     /// the read fails before the cancellation.
@@ -1053,6 +1180,61 @@ TEST_F(DiskIoMgrTest, VerifyNumThreadsParameter) {
   ASSERT_TRUE(num_io_threads ==
       num_io_threads_per_rotational_or_ssd + num_io_threads_for_remote_disks);
 }
+
+// Test to verify that the correct buffer sizes are chosen given different
+// of scan range lengths and max_bytes values.
+TEST_F(DiskIoMgrTest, BufferSizeSelection) {
+  DiskIoMgr io_mgr(1, 1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
+  ASSERT_OK(io_mgr.Init());
+
+  // Scan range doesn't fit in max_bytes - allocate as many max-sized buffers as possible.
+  EXPECT_EQ(vector<int64_t>(3, MAX_BUFFER_SIZE),
+      io_mgr.ChooseBufferSizes(10 * MAX_BUFFER_SIZE, 3 * MAX_BUFFER_SIZE));
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(10 * MAX_BUFFER_SIZE, MAX_BUFFER_SIZE));
+  EXPECT_EQ(vector<int64_t>(4, MAX_BUFFER_SIZE),
+      io_mgr.ChooseBufferSizes(10 * MAX_BUFFER_SIZE, 4 * MAX_BUFFER_SIZE));
+
+  // Scan range fits in max_bytes - allocate as many max-sized buffers as possible, then
+  // a smaller buffer to fit the remainder.
+  EXPECT_EQ(vector<int64_t>(2, MAX_BUFFER_SIZE),
+      io_mgr.ChooseBufferSizes(2 * MAX_BUFFER_SIZE, 3 * MAX_BUFFER_SIZE));
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE, MAX_BUFFER_SIZE, MIN_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(2 * MAX_BUFFER_SIZE + 1, 3 * MAX_BUFFER_SIZE));
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE, MAX_BUFFER_SIZE, 2 * MIN_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(
+        2 * MAX_BUFFER_SIZE + MIN_BUFFER_SIZE + 1, 3 * MAX_BUFFER_SIZE));
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE, MAX_BUFFER_SIZE, 2 * MIN_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(
+        2 * MAX_BUFFER_SIZE + 2 * MIN_BUFFER_SIZE, 3 * MAX_BUFFER_SIZE));
+
+  // Scan range is smaller than max buffer size - allocate a single buffer that fits
+  // the range.
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(MAX_BUFFER_SIZE - 1, 3 * MAX_BUFFER_SIZE));
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE / 2}),
+      io_mgr.ChooseBufferSizes(MAX_BUFFER_SIZE - 1, MAX_BUFFER_SIZE / 2));
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE / 2}),
+      io_mgr.ChooseBufferSizes(MAX_BUFFER_SIZE / 2 - 1, 3 * MAX_BUFFER_SIZE));
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE / 2}),
+      io_mgr.ChooseBufferSizes(MAX_BUFFER_SIZE / 2- 1, MAX_BUFFER_SIZE / 2));
+  EXPECT_EQ(vector<int64_t>({MIN_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(MIN_BUFFER_SIZE, 3 * MAX_BUFFER_SIZE));
+  EXPECT_EQ(vector<int64_t>({MIN_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(MIN_BUFFER_SIZE, MIN_BUFFER_SIZE));
+
+  // Scan range is smaller than max buffer size and max bytes is smaller still -
+  // should allocate a single smaller buffer.
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE / 4}),
+      io_mgr.ChooseBufferSizes(MAX_BUFFER_SIZE / 2, MAX_BUFFER_SIZE / 2 - 1));
+
+  // Non power-of-two size > max buffer size.
+  EXPECT_EQ(vector<int64_t>({MAX_BUFFER_SIZE, MIN_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(MAX_BUFFER_SIZE + 7, 3 * MAX_BUFFER_SIZE));
+  // Non power-of-two size < min buffer size.
+  EXPECT_EQ(vector<int64_t>({MIN_BUFFER_SIZE}),
+      io_mgr.ChooseBufferSizes(MIN_BUFFER_SIZE - 7, 3 * MAX_BUFFER_SIZE));
+}
 }
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/disk-io-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr.cc b/be/src/runtime/io/disk-io-mgr.cc
index 8c00ef8..6dda447 100644
--- a/be/src/runtime/io/disk-io-mgr.cc
+++ b/be/src/runtime/io/disk-io-mgr.cc
@@ -124,13 +124,6 @@ DEFINE_uint64(unused_file_handle_timeout_sec, 270, "Maximum time, in seconds, th
 DEFINE_uint64(num_file_handle_cache_partitions, 16, "Number of partitions used by the "
     "file handle cache.");
 
-// The IoMgr is able to run with a wide range of memory usage. If a query has memory
-// remaining less than this value, the IoMgr will stop all buffering regardless of the
-// current queue size.
-static const int LOW_MEMORY = 64 * 1024 * 1024;
-
-const int DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT;
-
 AtomicInt32 DiskIoMgr::next_disk_id_;
 
 namespace detail {
@@ -197,8 +190,8 @@ DiskIoMgr::DiskIoMgr() :
     num_io_threads_per_solid_state_disk_(GetFirstPositiveVal(
         FLAGS_num_io_threads_per_solid_state_disk, FLAGS_num_threads_per_disk,
         THREADS_PER_SOLID_STATE_DISK)),
-    max_buffer_size_(FLAGS_read_size),
-    min_buffer_size_(FLAGS_min_buffer_size),
+    max_buffer_size_(BitUtil::RoundUpToPowerOfTwo(FLAGS_read_size)),
+    min_buffer_size_(BitUtil::RoundDownToPowerOfTwo(FLAGS_min_buffer_size)),
     shut_down_(false),
     total_bytes_read_counter_(TUnit::BYTES),
     read_timer_(TUnit::TIME_NS),
@@ -223,8 +216,8 @@ DiskIoMgr::DiskIoMgr(int num_local_disks, int threads_per_rotational_disk,
     int threads_per_solid_state_disk, int64_t min_buffer_size, int64_t max_buffer_size) :
     num_io_threads_per_rotational_disk_(threads_per_rotational_disk),
     num_io_threads_per_solid_state_disk_(threads_per_solid_state_disk),
-    max_buffer_size_(max_buffer_size),
-    min_buffer_size_(min_buffer_size),
+    max_buffer_size_(BitUtil::RoundUpToPowerOfTwo(max_buffer_size)),
+    min_buffer_size_(BitUtil::RoundDownToPowerOfTwo(min_buffer_size)),
     shut_down_(false),
     total_bytes_read_counter_(TUnit::BYTES),
     read_timer_(TUnit::TIME_NS),
@@ -335,80 +328,90 @@ Status DiskIoMgr::ValidateScanRange(ScanRange* range) {
     return Status(TErrorCode::DISK_IO_ERROR,
         Substitute("Invalid scan range. Negative offset $0", range->offset_));
   }
-  if (range->len_ < 0) {
+  if (range->len_ <= 0) {
     return Status(TErrorCode::DISK_IO_ERROR,
-        Substitute("Invalid scan range. Negative length $0", range->len_));
+        Substitute("Invalid scan range. Non-positive length $0", range->len_));
   }
   return Status::OK();
 }
 
-Status DiskIoMgr::AddScanRanges(RequestContext* reader,
-    const vector<ScanRange*>& ranges, bool schedule_immediately) {
-  if (ranges.empty()) return Status::OK();
-
+Status DiskIoMgr::AddScanRanges(
+    RequestContext* reader, const vector<ScanRange*>& ranges) {
   // Validate and initialize all ranges
   for (int i = 0; i < ranges.size(); ++i) {
     RETURN_IF_ERROR(ValidateScanRange(ranges[i]));
     ranges[i]->InitInternal(this, reader);
   }
 
-  // disks that this reader needs to be scheduled on.
   unique_lock<mutex> reader_lock(reader->lock_);
   DCHECK(reader->Validate()) << endl << reader->DebugString();
 
   if (reader->state_ == RequestContext::Cancelled) return Status::CANCELLED;
 
   // Add each range to the queue of the disk the range is on
-  for (int i = 0; i < ranges.size(); ++i) {
+  for (ScanRange* range : ranges) {
     // Don't add empty ranges.
-    DCHECK_NE(ranges[i]->len(), 0);
-    ScanRange* range = ranges[i];
-
+    DCHECK_NE(range->len(), 0);
+    reader->AddActiveScanRangeLocked(reader_lock, range);
     if (range->try_cache_) {
-      if (schedule_immediately) {
-        bool cached_read_succeeded;
-        RETURN_IF_ERROR(range->ReadFromCache(reader_lock, &cached_read_succeeded));
-        if (cached_read_succeeded) continue;
-        // Cached read failed, fall back to AddRequestRange() below.
-      } else {
-        reader->cached_ranges_.Enqueue(range);
-        continue;
-      }
+      reader->cached_ranges_.Enqueue(range);
+    } else {
+      reader->AddRangeToDisk(reader_lock, range, ScheduleMode::UPON_GETNEXT);
     }
-    reader->AddRequestRange(reader_lock, range, schedule_immediately);
   }
   DCHECK(reader->Validate()) << endl << reader->DebugString();
-
   return Status::OK();
 }
 
-Status DiskIoMgr::AddScanRange(
-    RequestContext* reader, ScanRange* range, bool schedule_immediately) {
-  return AddScanRanges(reader, vector<ScanRange*>({range}), schedule_immediately);
+Status DiskIoMgr::StartScanRange(RequestContext* reader, ScanRange* range,
+    bool* needs_buffers) {
+  RETURN_IF_ERROR(ValidateScanRange(range));
+  range->InitInternal(this, reader);
+
+  unique_lock<mutex> reader_lock(reader->lock_);
+  DCHECK(reader->Validate()) << endl << reader->DebugString();
+  if (reader->state_ == RequestContext::Cancelled) return Status::CANCELLED;
+
+  DCHECK_NE(range->len(), 0);
+  if (range->try_cache_) {
+    bool cached_read_succeeded;
+    RETURN_IF_ERROR(range->ReadFromCache(reader_lock, &cached_read_succeeded));
+    if (cached_read_succeeded) {
+      DCHECK(reader->Validate()) << endl << reader->DebugString();
+      *needs_buffers = false;
+      return Status::OK();
+    }
+    // Cached read failed, fall back to normal read path.
+  }
+  // If we don't have a buffer yet, the caller must allocate buffers for the range.
+  *needs_buffers = range->external_buffer_tag_ == ScanRange::ExternalBufferTag::NO_BUFFER;
+  if (*needs_buffers) range->SetBlockedOnBuffer();
+  reader->AddActiveScanRangeLocked(reader_lock, range);
+  reader->AddRangeToDisk(reader_lock, range,
+      *needs_buffers ? ScheduleMode::BY_CALLER : ScheduleMode::IMMEDIATELY);
+  DCHECK(reader->Validate()) << endl << reader->DebugString();
+  return Status::OK();
 }
 
 // This function returns the next scan range the reader should work on, checking
 // for eos and error cases. If there isn't already a cached scan range or a scan
 // range prepared by the disk threads, the caller waits on the disk threads.
-Status DiskIoMgr::GetNextRange(RequestContext* reader, ScanRange** range) {
+Status DiskIoMgr::GetNextUnstartedRange(RequestContext* reader, ScanRange** range,
+    bool* needs_buffers) {
   DCHECK(reader != nullptr);
   DCHECK(range != nullptr);
   *range = nullptr;
-  Status status = Status::OK();
+  *needs_buffers = false;
 
   unique_lock<mutex> reader_lock(reader->lock_);
   DCHECK(reader->Validate()) << endl << reader->DebugString();
-
   while (true) {
-    if (reader->state_ == RequestContext::Cancelled) {
-      status = Status::CANCELLED;
-      break;
-    }
+    if (reader->state_ == RequestContext::Cancelled) return Status::CANCELLED;
 
     if (reader->num_unstarted_scan_ranges_.Load() == 0 &&
         reader->ready_to_start_ranges_.empty() && reader->cached_ranges_.empty()) {
       // All ranges are done, just return.
-      break;
+      return Status::OK();
     }
 
     if (!reader->cached_ranges_.empty()) {
@@ -420,7 +423,7 @@ Status DiskIoMgr::GetNextRange(RequestContext* reader, ScanRange** range) {
       if (cached_read_succeeded) return Status::OK();
 
       // This range ended up not being cached. Loop again and pick up a new range.
-      reader->AddRequestRange(reader_lock, *range, false);
+      reader->AddRangeToDisk(reader_lock, *range, ScheduleMode::UPON_GETNEXT);
       DCHECK(reader->Validate()) << endl << reader->DebugString();
       *range = nullptr;
       continue;
@@ -436,13 +439,81 @@ Status DiskIoMgr::GetNextRange(RequestContext* reader, ScanRange** range) {
       // Set this to nullptr, the next time this disk runs for this reader, it will
       // get another range ready.
       reader->disk_states_[disk_id].set_next_scan_range_to_start(nullptr);
-      reader->ScheduleScanRange(reader_lock, *range);
-      break;
+      ScanRange::ExternalBufferTag buffer_tag = (*range)->external_buffer_tag_;
+      if (buffer_tag == ScanRange::ExternalBufferTag::NO_BUFFER) {
+        // We can't schedule this range until the client gives us buffers. The context
+        // must be rescheduled regardless to ensure that 'next_scan_range_to_start' is
+        // refilled.
+        reader->disk_states_[disk_id].ScheduleContext(reader_lock, reader, disk_id);
+        (*range)->SetBlockedOnBuffer();
+        *needs_buffers = true;
+      } else {
+        reader->ScheduleScanRange(reader_lock, *range);
+      }
+      return Status::OK();
+    }
+  }
+}
+
+Status DiskIoMgr::AllocateBuffersForRange(RequestContext* reader, ScanRange* range,
+    int64_t max_bytes) {
+  DCHECK_GE(max_bytes, min_buffer_size_);
+  DCHECK(range->external_buffer_tag_ == ScanRange::ExternalBufferTag::NO_BUFFER)
+     << static_cast<int>(range->external_buffer_tag_) << " invalid to allocate buffers "
+     << "when already reading into an external buffer";
+
+  Status status;
+  vector<unique_ptr<BufferDescriptor>> buffers;
+  for (int64_t buffer_size : ChooseBufferSizes(range->len(), max_bytes)) {
+    if (!reader->mem_tracker_->TryConsume(buffer_size)) {
+      status = reader->mem_tracker_->MemLimitExceeded(nullptr,
+          "Failed to allocate I/O buffer", buffer_size);
+      goto error;
     }
+    uint8_t* buffer = reinterpret_cast<uint8_t*>(malloc(buffer_size));
+    if (buffer == nullptr) {
+      reader->mem_tracker_->Release(buffer_size);
+      status = Status(Substitute("Failed to malloc $0-byte I/O buffer", buffer_size));
+      goto error;
+    }
+    buffers.emplace_back(new BufferDescriptor(this, reader, range, buffer, buffer_size));
   }
+  range->AddUnusedBuffers(move(buffers), false);
+  return Status::OK();
+ error:
+  DCHECK(!status.ok());
+  range->CleanUpBuffers(move(buffers));
   return status;
 }
 
+vector<int64_t> DiskIoMgr::ChooseBufferSizes(int64_t scan_range_len, int64_t max_bytes) {
+  DCHECK_GE(max_bytes, min_buffer_size_);
+  vector<int64_t> buffer_sizes;
+  int64_t bytes_allocated = 0;
+  while (bytes_allocated < scan_range_len) {
+    int64_t bytes_remaining = scan_range_len - bytes_allocated;
+    // Either allocate a max-sized buffer or a smaller buffer to fit the rest of the
+    // range.
+    int64_t next_buffer_size;
+    if (bytes_remaining >= max_buffer_size_) {
+      next_buffer_size = max_buffer_size_;
+    } else {
+      next_buffer_size =
+          max(min_buffer_size_, BitUtil::RoundUpToPowerOfTwo(bytes_remaining));
+    }
+    if (next_buffer_size + bytes_allocated > max_bytes) {
+      // Can't allocate the desired buffer size. Make sure to allocate at least one
+      // buffer.
+      if (bytes_allocated > 0) break;
+      next_buffer_size = BitUtil::RoundDownToPowerOfTwo(max_bytes);
+    }
+    DCHECK(BitUtil::IsPowerOf2(next_buffer_size)) << next_buffer_size;
+    buffer_sizes.push_back(next_buffer_size);
+    bytes_allocated += next_buffer_size;
+  }
+  return buffer_sizes;
+}
+
 // This function gets the next RequestRange to work on for this disk. It checks for
 // cancellation and
 // a) Updates ready_to_start_ranges if there are no scan ranges queued for this disk.
@@ -505,16 +576,16 @@ bool DiskIoMgr::GetNextRequestRange(DiskQueue* disk_queue, RequestRange** range,
         !request_disk_state->unstarted_scan_ranges()->empty()) {
       // We don't have a range queued for this disk for what the caller should
       // read next. Populate that.  We want to have one range waiting to minimize
-      // wait time in GetNextRange.
+      // wait time in GetNextUnstartedRange().
       ScanRange* new_range = request_disk_state->unstarted_scan_ranges()->Dequeue();
       (*request_context)->num_unstarted_scan_ranges_.Add(-1);
       (*request_context)->ready_to_start_ranges_.Enqueue(new_range);
       request_disk_state->set_next_scan_range_to_start(new_range);
 
       if ((*request_context)->num_unstarted_scan_ranges_.Load() == 0) {
-        // All the ranges have been started, notify everyone blocked on GetNextRange.
-        // Only one of them will get work so make sure to return nullptr to the other
-        // caller threads.
+        // All the ranges have been started, notify everyone blocked on
+        // GetNextUnstartedRange(). Only one of them will get work so make sure to return
+        // nullptr to the other caller threads.
         (*request_context)->ready_to_start_ranges_cv_.NotifyAll();
       } else {
         (*request_context)->ready_to_start_ranges_cv_.NotifyOne();
@@ -584,8 +655,8 @@ void DiskIoMgr::HandleReadFinished(DiskQueue* disk_queue, RequestContext* reader
   DCHECK(buffer->buffer_ != nullptr);
   DCHECK(!buffer->is_cached()) << "HDFS cache reads don't go through this code path.";
 
-  // After calling EnqueueBuffer() below, it is no longer valid to read from buffer.
-  // Store the state we need before calling EnqueueBuffer().
+  // After calling EnqueueReadyBuffer() below, it is no longer valid to read from buffer.
+  // Store the state we need before calling EnqueueReadyBuffer().
   bool eosr = buffer->eosr_;
 
   // TODO: IMPALA-4249: it safe to touch 'scan_range' until DecrementDiskThread() is
@@ -596,23 +667,18 @@ void DiskIoMgr::HandleReadFinished(DiskQueue* disk_queue, RequestContext* reader
   if (read_status.ok() && reader->state_ != RequestContext::Cancelled) {
     DCHECK_EQ(reader->state_, RequestContext::Active);
     // Read successfully - update the reader's scan ranges.  There are two cases here:
-    //  1. End of scan range
-    //  2. Middle of scan range
-    bool queue_full = scan_range->EnqueueBuffer(reader_lock, move(buffer));
-    if (!eosr) {
-      if (queue_full) {
-        reader->blocked_ranges_.Enqueue(scan_range);
-      } else {
-        reader->ScheduleScanRange(reader_lock, scan_range);
-      }
-    }
+    //  1. End of scan range or cancelled scan range - don't need to reschedule.
+    //  2. Middle of scan range - need to schedule to read next buffer.
+    bool enqueued = scan_range->EnqueueReadyBuffer(reader_lock, move(buffer));
+    if (!eosr && enqueued) reader->ScheduleScanRange(reader_lock, scan_range);
   } else {
     // The scan range will be cancelled, either because we hit an error or because the
     // request context was cancelled.  The buffer is not needed - we must free it.
     reader->FreeBuffer(buffer.get());
-    reader->num_used_buffers_.Add(-1);
-    // Propagate the error or cancellation by cancelling the scan range.
-    scan_range->Cancel(read_status.ok() ? Status::CANCELLED : read_status);
+    // Propagate 'read_status' to the scan range. If we are here because the context
+    // was cancelled, the scan range is already cancelled so we do not need to re-cancel
+    // it.
+    if (!read_status.ok()) scan_range->CancelFromReader(reader_lock, read_status);
     scan_range_done = true;
   }
   if (scan_range_done) {
@@ -636,14 +702,12 @@ void DiskIoMgr::WorkLoop(DiskQueue* disk_queue) {
   //   3. Perform the read or write as specified.
   // Cancellation checking needs to happen in both steps 1 and 3.
   while (true) {
-    RequestContext* worker_context = nullptr;;
+    RequestContext* worker_context = nullptr;
     RequestRange* range = nullptr;
-
     if (!GetNextRequestRange(disk_queue, &range, &worker_context)) {
       DCHECK(shut_down_);
-      break;
+      return;
     }
-
     if (range->request_type() == RequestType::READ) {
       ReadRange(disk_queue, worker_context, static_cast<ScanRange*>(range));
     } else {
@@ -651,12 +715,8 @@ void DiskIoMgr::WorkLoop(DiskQueue* disk_queue) {
       Write(worker_context, static_cast<WriteRange*>(range));
     }
   }
-
-  DCHECK(shut_down_);
 }
 
-// This function reads the specified scan range associated with the
-// specified reader context and disk queue.
 void DiskIoMgr::ReadRange(
     DiskQueue* disk_queue, RequestContext* reader, ScanRange* range) {
   int64_t bytes_remaining = range->len_ - range->bytes_read_;
@@ -669,25 +729,23 @@ void DiskIoMgr::ReadRange(
     DCHECK(range->external_buffer_tag_ == ScanRange::ExternalBufferTag::NO_BUFFER)
         << "This code path does not handle other buffer types, i.e. HDFS cache"
         << static_cast<int>(range->external_buffer_tag_);
-    // Need to allocate a buffer to read into.
-    int64_t buffer_size = min(bytes_remaining, max_buffer_size_);
-    buffer_desc = TryAllocateNextBufferForRange(disk_queue, reader, range, buffer_size);
-    if (buffer_desc == nullptr) return;
+    buffer_desc = range->GetNextUnusedBufferForRange();
+    if (buffer_desc == nullptr) {
+      // No buffer available - the range will be rescheduled when a buffer is added.
+      unique_lock<mutex> reader_lock(reader->lock_);
+      reader->disk_states_[disk_queue->disk_id].DecrementDiskThread(reader_lock, reader);
+      DCHECK(reader->Validate()) << endl << reader->DebugString();
+      return;
+    }
   }
-  reader->num_used_buffers_.Add(1);
 
   // No locks in this section.  Only working on local vars.  We don't want to hold a
   // lock across the read call.
   Status read_status = range->Open(detail::is_file_handle_caching_enabled());
   if (read_status.ok()) {
     // Update counters.
-    if (reader->active_read_thread_counter_) {
-      reader->active_read_thread_counter_->Add(1L);
-    }
-    if (reader->disks_accessed_bitmap_) {
-      int64_t disk_bit = 1LL << disk_queue->disk_id;
-      reader->disks_accessed_bitmap_->BitOr(disk_bit);
-    }
+    COUNTER_ADD_IF_NOT_NULL(reader->active_read_thread_counter_, 1L);
+    COUNTER_BITOR_IF_NOT_NULL(reader->disks_accessed_bitmap_, 1LL << disk_queue->disk_id);
     SCOPED_TIMER(&read_timer_);
     SCOPED_TIMER(reader->read_timer_);
 
@@ -695,69 +753,15 @@ void DiskIoMgr::ReadRange(
         &buffer_desc->len_, &buffer_desc->eosr_);
     buffer_desc->scan_range_offset_ = range->bytes_read_ - buffer_desc->len_;
 
-    if (reader->bytes_read_counter_ != nullptr) {
-      COUNTER_ADD(reader->bytes_read_counter_, buffer_desc->len_);
-    }
-
+    COUNTER_ADD_IF_NOT_NULL(reader->bytes_read_counter_, buffer_desc->len_);
     COUNTER_ADD(&total_bytes_read_counter_, buffer_desc->len_);
-    if (reader->active_read_thread_counter_) {
-      reader->active_read_thread_counter_->Add(-1L);
-    }
+    COUNTER_ADD_IF_NOT_NULL(reader->active_read_thread_counter_, -1L);
   }
 
   // Finished read, update reader/disk based on the results
   HandleReadFinished(disk_queue, reader, read_status, move(buffer_desc));
 }
 
-unique_ptr<BufferDescriptor> DiskIoMgr::TryAllocateNextBufferForRange(
-    DiskQueue* disk_queue, RequestContext* reader, ScanRange* range,
-    int64_t buffer_size) {
-  DCHECK(reader->mem_tracker_ != nullptr);
-  // TODO: replace this with reservation check (if needed at all).
-  bool enough_memory = reader->mem_tracker_->SpareCapacity() > LOW_MEMORY;
-
-  RequestContext::PerDiskState* disk_state = &reader->disk_states_[disk_queue->disk_id];
-  if (!enough_memory) {
-    unique_lock<mutex> reader_lock(reader->lock_);
-
-    // Just grabbed the reader lock, check for cancellation.
-    if (reader->state_ == RequestContext::Cancelled) {
-      DCHECK(reader->Validate()) << endl << reader->DebugString();
-      disk_state->DecrementDiskThread(reader_lock, reader);
-      range->Cancel(Status::CANCELLED);
-      DCHECK(reader->Validate()) << endl << reader->DebugString();
-      return nullptr;
-    }
-
-    if (!range->ready_buffers_.empty()) {
-      // We have memory pressure and this range doesn't need another buffer
-      // (it already has one queued). Skip this range and pick it up later.
-      range->blocked_on_queue_ = true;
-      reader->blocked_ranges_.Enqueue(range);
-      disk_state->DecrementDiskThread(reader_lock, reader);
-      return nullptr;
-    } else {
-      // We need to get a buffer anyway since there are none queued. The query
-      // is likely to fail due to mem limits but there's nothing we can do about that
-      // now.
-    }
-  }
-  unique_ptr<BufferDescriptor> buffer_desc;
-  Status status = reader->AllocBuffer(range, buffer_size, &buffer_desc);
-  if (!status.ok()) {
-    // Hit memory limit - cancel range.
-    range->Cancel(status);
-    {
-      unique_lock<mutex> reader_lock(reader->lock_);
-      disk_state->DecrementDiskThread(reader_lock, reader);
-      DCHECK(reader->Validate()) << endl << reader->DebugString();
-    }
-    return nullptr;
-  }
-  DCHECK(buffer_desc != nullptr);
-  return buffer_desc;
-}
-
 void DiskIoMgr::Write(RequestContext* writer_context, WriteRange* write_range) {
   Status ret_status = Status::OK();
   FILE* file_handle = nullptr;
@@ -809,17 +813,14 @@ Status DiskIoMgr::WriteRangeHelper(FILE* file_handle, WriteRange* write_range) {
         Substitute("fwrite(buffer, 1, $0, $1) failed with errno=$2 description=$3",
         write_range->len_, write_range->file_, errno, GetStrErrMsg())));
   }
-  if (ImpaladMetrics::IO_MGR_BYTES_WRITTEN != nullptr) {
-    ImpaladMetrics::IO_MGR_BYTES_WRITTEN->Increment(write_range->len_);
-  }
-
+  ImpaladMetrics::IO_MGR_BYTES_WRITTEN->Increment(write_range->len_);
   return Status::OK();
 }
 
 Status DiskIoMgr::AddWriteRange(RequestContext* writer, WriteRange* write_range) {
   unique_lock<mutex> writer_lock(writer->lock_);
   if (writer->state_ == RequestContext::Cancelled) return Status::CANCELLED;
-  writer->AddRequestRange(writer_lock, write_range, false);
+  writer->AddRangeToDisk(writer_lock, write_range, ScheduleMode::IMMEDIATELY);
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/disk-io-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr.h b/be/src/runtime/io/disk-io-mgr.h
index 20f5d44..d429d1d 100644
--- a/be/src/runtime/io/disk-io-mgr.h
+++ b/be/src/runtime/io/disk-io-mgr.h
@@ -48,15 +48,15 @@ namespace io {
 /// Manager object that schedules IO for all queries on all disks and remote filesystems
 /// (such as S3). Each query maps to one or more RequestContext objects, each of which
 /// has its own queue of scan ranges and/or write ranges.
-//
+///
 /// The API splits up requesting scan/write ranges (non-blocking) and reading the data
 /// (blocking). The DiskIoMgr has worker threads that will read from and write to
 /// disk/hdfs/remote-filesystems, allowing interleaving of IO and CPU. This allows us to
 /// keep all disks and all cores as busy as possible.
-//
+///
 /// All public APIs are thread-safe. It is not valid to call any of the APIs after
 /// UnregisterContext() returns.
-//
+///
 /// For Readers:
 /// We can model this problem as a multiple producer (threads for each disk), multiple
 /// consumer (scan ranges) problem. There are multiple queues that need to be
@@ -68,84 +68,101 @@ namespace io {
 /// Readers map to scan nodes. The reader then contains a queue of scan ranges. The caller
 /// asks the IoMgr for the next range to process. The IoMgr then selects the best range
 /// to read based on disk activity and begins reading and queuing buffers for that range.
-/// TODO: We should map readers to queries. A reader is the unit of scheduling and queries
-/// that have multiple scan nodes shouldn't have more 'turns'.
-//
+///
 /// For Writers:
 /// Data is written via AddWriteRange(). This is non-blocking and adds a WriteRange to a
 /// per-disk queue. After the write is complete, a callback in WriteRange is invoked.
 /// No memory is allocated within IoMgr for writes and no copies are made. It is the
 /// responsibility of the client to ensure that the data to be written is valid and that
 /// the file to be written to exists until the callback is invoked.
-//
-/// The IoMgr provides three key APIs.
-///  1. AddScanRanges: this is non-blocking and tells the IoMgr all the ranges that
-///     will eventually need to be read.
-///  2. GetNextRange: returns to the caller the next scan range it should process.
-///     This is based on disk load. This also begins reading the data in this scan
-///     range. This is blocking.
-///  3. ScanRange::GetNext: returns the next buffer for this range.  This is blocking.
-//
+///
+/// There are several key methods for scanning data with the IoMgr.
+///  1. StartScanRange(): adds range to the IoMgr to start immediately.
+///  2. AddScanRanges(): adds ranges to the IoMgr that the reader wants to scan, but does
+///     not start them until GetNextUnstartedRange() is called.
+///  3. GetNextUnstartedRange(): returns to the caller the next scan range it should
+///     process.
+///  4. ScanRange::GetNext(): returns the next buffer for this range, blocking until
+///     data is available.
+///
 /// The disk threads do not synchronize with each other. The readers and writers don't
 /// synchronize with each other. There is a lock and condition variable for each request
 /// context queue and each disk queue.
 /// IMPORTANT: whenever both locks are needed, the lock order is to grab the context lock
 /// before the disk lock.
-//
+///
 /// Scheduling: If there are multiple request contexts with work for a single disk, the
 /// request contexts are scheduled in round-robin order. Multiple disk threads can
 /// operate on the same request context. Exactly one request range is processed by a
-/// disk thread at a time. If there are multiple scan ranges scheduled via
-/// GetNextRange() for a single context, these are processed in round-robin order.
+/// disk thread at a time. If there are multiple scan ranges scheduled for a single
+/// context, these are processed in round-robin order.
 /// If there are multiple scan and write ranges for a disk, a read is always followed
 /// by a write, and a write is followed by a read, i.e. reads and writes alternate.
 /// If multiple write ranges are enqueued for a single disk, they will be processed
 /// by the disk threads in order, but may complete in any order. No guarantees are made
 /// on ordering of writes across disks.
-//
-/// Resource Management: effective resource management in the IoMgr is key to good
-/// performance. The IoMgr helps coordinate two resources: CPU and disk. For CPU,
-/// spinning up too many threads causes thrashing.
-/// Memory usage in the IoMgr comes from queued read buffers.  If we queue the minimum
-/// (i.e. 1), then the disks are idle while we are processing the buffer. If we don't
-/// limit the queue, then it possible we end up queueing the entire data set (i.e. CPU
-/// is slower than disks) and run out of memory.
-/// For both CPU and memory, we want to model the machine as having a fixed amount of
-/// resources.  If a single query is running, it should saturate either CPU or Disk
-/// as well as using as little memory as possible. With multiple queries, each query
-/// should get less CPU. In that case each query will need fewer queued buffers and
-/// therefore have less memory usage.
-//
-/// The IoMgr defers CPU management to the caller. The IoMgr provides a GetNextRange
-/// API which will return the next scan range the caller should process. The caller
-/// can call this from the desired number of reading threads. Once a scan range
-/// has been returned via GetNextRange, the IoMgr will start to buffer reads for
-/// that range and it is expected the caller will pull those buffers promptly. For
-/// example, if the caller would like to have 1 scanner thread, the read loop
-/// would look like:
+///
+/// Resource Management: the IoMgr is designed to share the available disk I/O capacity
+/// between many clients and to help use the available I/O capacity efficiently. The IoMgr
+/// interfaces are designed to let clients manage their own CPU and memory usage while the
+/// IoMgr manages the allocation of the I/O capacity of different I/O devices to scan
+/// ranges of different clients.
+///
+/// IoMgr clients may want to work on multiple scan ranges at a time to maximize CPU and
+/// I/O utilization. Clients can call GetNextUnstartedRange() to start as many concurrent
+/// scan ranges as required, e.g. from each parallel scanner thread. Once a scan range has
+/// been returned via GetNextUnstartedRange(), the caller must allocate any memory needed
+/// for buffering reads, after which the IoMgr wil start to fill the buffers with data
+/// while the caller concurrently consumes and processes the data. For example, the logic
+/// in a scanner thread might look like:
 ///   while (more_ranges)
-///     range = GetNextRange()
+///     range = GetNextUnstartedRange()
 ///     while (!range.eosr)
 ///       buffer = range.GetNext()
-/// To have multiple reading threads, the caller would simply spin up the threads
-/// and each would process the loops above.
-//
-/// To control the number of IO buffers, each scan range has a limit of two queued
-/// buffers (SCAN_RANGE_READY_BUFFER_LIMIT). If the number of buffers is at capacity,
-/// the IoMgr will no longer read for that scan range until the caller has processed
-/// a buffer. Assuming the client returns each buffer before requesting the next one
-/// from the scan range, then this will consume up to 3 * 8MB = 24MB of I/O buffers per
-/// scan range.
-//
+///
+/// Note that the IoMgr rather than the client is responsible for choosing which scan
+/// range to process next, which allows optimizations like distributing load across disks.
+///
 /// Buffer Management:
-/// Buffers for reads are either a) allocated by the IoMgr and transferred to the caller,
-/// b) cached HDFS buffers if the scan range uses HDFS caching, or c) provided by the
-/// caller when constructing the scan range.
+/// Buffers for reads are either a) allocated on behalf of the caller with
+/// AllocateBuffersForRange() ("IoMgr-allocated"), b) cached HDFS buffers if the scan
+/// range was read from the HDFS cache, or c) a client buffer, large enough to fit the
+/// whole scan range's data, that is provided by the caller when constructing the
+/// scan range.
+///
+/// All three kinds of buffers are wrapped in BufferDescriptors before returning to the
+/// caller. The caller must always call ReturnBuffer() on the buffer descriptor to allow
+/// recycling of the buffer memory and to release any resources associated with the buffer
+/// or scan range.
 ///
-/// As a caller reads from a scan range, these buffers are wrapped in BufferDescriptors
-/// and returned to the caller. The caller must always call ReturnBuffer() on the buffer
-/// descriptor to allow freeing of the associated buffer (if there is an IoMgr-allocated
-/// or HDFS cached buffer).
+/// In case a), ReturnBuffer() may re-enqueue the buffer for GetNext() to return again if
+/// needed. E.g. if 24MB of buffers were allocated to read a 64MB scan range, each buffer
+/// must be returned multiple times. Callers must be careful to call ReturnBuffer() with
+/// the previous buffer returned from the range before calling before GetNext() so that
+/// at least one buffer is available for the I/O mgr to read data into. Calling GetNext()
+/// when the scan range has no buffers to read data into causes a resource deadlock.
+/// NB: if the scan range was allocated N buffers, then it's always ok for the caller
+/// to hold onto N - 1 buffers, but currently the IoMgr doesn't give the caller a way
+/// to determine the value of N.
+///
+/// If the caller wants to maximize I/O throughput, it can give the range enough memory
+/// for 3 max-sized buffers per scan range. Having two queued buffers (plus the buffer
+/// that is currently being processed by the client) gives good performance in most
+/// scenarios:
+/// 1. If the consumer is consuming data faster than we can read from disk, then the
+///    queue will be empty most of the time because the buffer will be immediately
+///    pulled off the queue as soon as it is added. There will always be an I/O request
+///    in the disk queue to maximize I/O throughput, which is the bottleneck in this
+///    case.
+/// 2. If we can read from disk faster than the consumer is consuming data, the queue
+///    will fill up and there will always be a buffer available for the consumer to
+///    read, so the consumer will not block and we maximize consumer throughput, which
+///    is the bottleneck in this case.
+/// 3. If the consumer is consuming data at approximately the same rate as we are
+///    reading from disk, then the steady state is that the consumer is processing one
+///    buffer and one buffer is in the disk queue. The additional buffer can absorb
+///    bursts where the producer runs faster than the consumer or the consumer runs
+///    faster than the producer without blocking either the producer or consumer.
 ///
 /// Caching support:
 /// Scan ranges contain metadata on whether or not it is cached on the DN. In that
@@ -161,13 +178,13 @@ namespace io {
 ///   - HDFS will time us out if we hold onto the mlock for too long
 ///   - Holding the lock prevents uncaching this file due to a caching policy change.
 /// Therefore, we only issue the cached read when the caller is ready to process the
-/// range (GetNextRange()) instead of when the ranges are issued. This guarantees that
-/// there will be a CPU available to process the buffer and any throttling we do with
+/// range (GetNextUnstartedRange()) instead of when the ranges are issued. This guarantees
+/// that there will be a CPU available to process the buffer and any throttling we do with
 /// the number of scanner threads properly controls the amount of files we mlock.
 /// With cached scan ranges, we cannot close the scan range until the cached buffer
 /// is returned (HDFS does not allow this). We therefore need to defer the close until
 /// the cached buffer is returned (ReturnBuffer()).
-//
+///
 /// Remote filesystem support (e.g. S3):
 /// Remote filesystems are modeled as "remote disks". That is, there is a seperate disk
 /// queue for each supported remote filesystem type. In order to maximize throughput,
@@ -176,12 +193,13 @@ namespace io {
 /// intensive than local disk/hdfs because of non-direct I/O and SSL processing, and can
 /// be CPU bottlenecked especially if not enough I/O threads for these queues are
 /// started.
-//
+///
+/// TODO: We should implement more sophisticated resource management. Currently readers
+/// are the unit of scheduling and we attempt to distribute IOPS between them. Instead
+/// it would be better to have policies based on queries, resource pools, etc.
 /// TODO: IoMgr should be able to request additional scan ranges from the coordinator
 /// to help deal with stragglers.
-/// TODO: look into using a lock free queue
-/// TODO: simplify the common path (less locking, memory allocations).
-//
+///
 /// Structure of the Implementation:
 ///  - All client APIs are defined in this file, request-ranges.h and request-context.h.
 ///    Clients can include only the files that they need.
@@ -204,8 +222,10 @@ class DiskIoMgr : public CacheLineAligned {
   ///    disk. This is also the max queue depth.
   ///  - threads_per_solid_state_disk: number of read threads to create per solid state
   ///    disk. This is also the max queue depth.
-  ///  - min_buffer_size: minimum io buffer size (in bytes)
-  ///  - max_buffer_size: maximum io buffer size (in bytes). Also the max read size.
+  ///  - min_buffer_size: minimum io buffer size (in bytes). Will be rounded down to the
+  //     nearest power-of-two.
+  ///  - max_buffer_size: maximum io buffer size (in bytes). Will be rounded up to the
+  ///    nearest power-of-two. Also the max read size.
   DiskIoMgr(int num_disks, int threads_per_rotational_disk,
       int threads_per_solid_state_disk, int64_t min_buffer_size, int64_t max_buffer_size);
 
@@ -237,29 +257,61 @@ class DiskIoMgr : public CacheLineAligned {
   /// up.
   void UnregisterContext(RequestContext* context);
 
-  /// Adds the scan ranges to the queues. This call is non-blocking. The caller must
-  /// not deallocate the scan range pointers before UnregisterContext().
-  /// If schedule_immediately, the ranges are immediately put on the read queue
-  /// (i.e. the caller should not/cannot call GetNextRange for these ranges).
-  /// This can be used to do synchronous reads as well as schedule dependent ranges,
-  /// as in the case for columnar formats.
-  Status AddScanRanges(RequestContext* reader,
-      const std::vector<ScanRange*>& ranges,
-      bool schedule_immediately = false) WARN_UNUSED_RESULT;
-  Status AddScanRange(RequestContext* reader, ScanRange* range,
-      bool schedule_immediately = false) WARN_UNUSED_RESULT;
+  /// Adds the scan ranges to reader's queues, but does not start scheduling it. The range
+  /// can be scheduled by a thread calling GetNextUnstartedRange(). This call is
+  /// non-blocking. The caller must not deallocate the scan range pointers before
+  /// UnregisterContext().
+  Status AddScanRanges(
+      RequestContext* reader, const std::vector<ScanRange*>& ranges) WARN_UNUSED_RESULT;
+
+  /// Adds the scan range to the queues, as with AddScanRanges(), but immediately
+  /// start scheduling the scan range. This can be used to do synchronous reads as well
+  /// as schedule dependent ranges, e.g. for columnar formats. This call is non-blocking.
+  /// The caller must not deallocate the scan range pointers before UnregisterContext().
+  ///
+  /// If this returns true in '*needs_buffers', the caller must then call
+  /// AllocateBuffersForRange() to add buffers for the data to be read into before the
+  /// range can be scheduled. Otherwise, the range is scheduled and the IoMgr will
+  /// asynchronously read the data for the range and the caller can call
+  /// ScanRange::GetNext() to read the data.
+  Status StartScanRange(
+      RequestContext* reader, ScanRange* range, bool* needs_buffers) WARN_UNUSED_RESULT;
 
   /// Add a WriteRange for the writer. This is non-blocking and schedules the context
   /// on the IoMgr disk queue. Does not create any files.
   Status AddWriteRange(
       RequestContext* writer, WriteRange* write_range) WARN_UNUSED_RESULT;
 
-  /// Returns the next unstarted scan range for this reader. When the range is returned,
-  /// the disk threads in the IoMgr will already have started reading from it. The
-  /// caller is expected to call ScanRange::GetNext on the returned range.
-  /// If there are no more unstarted ranges, nullptr is returned.
-  /// This call is blocking.
-  Status GetNextRange(RequestContext* reader, ScanRange** range) WARN_UNUSED_RESULT;
+  /// Tries to get an unstarted scan range that was added to 'reader' with
+  /// AddScanRanges(). On success, returns OK and returns the range in '*range'.
+  /// If 'reader' was cancelled, returns CANCELLED. If another error is encountered,
+  /// an error status is returned. Otherwise, if error or cancellation wasn't encountered
+  /// and there are no unstarted ranges for 'reader', returns OK and sets '*range' to
+  /// nullptr.
+  ///
+  /// If '*needs_buffers' is returned as true, the caller must call
+  /// AllocateBuffersForRange() to add buffers for the data to be read into before the
+  /// range can be scheduled. Otherwise, the range is scheduled and the IoMgr will
+  /// asynchronously read the data for the range and the caller can call
+  /// ScanRange::GetNext() to read the data.
+  Status GetNextUnstartedRange(RequestContext* reader, ScanRange** range,
+      bool* needs_buffers) WARN_UNUSED_RESULT;
+
+  /// Allocates up to 'max_bytes' buffers to read the data from 'range' into and schedules
+  /// the range. Called after StartScanRange() or GetNextUnstartedRange() returns
+  /// *needs_buffers=true.
+  ///
+  /// The buffer sizes are chosen based on range->len(). 'max_bytes' must be >=
+  /// min_read_buffer_size() so that at least one buffer can be allocated. Returns ok
+  /// if the buffers were successfully allocated and the range was scheduled. Fails with
+  /// MEM_LIMIT_EXCEEDED if the buffers could not be allocated. On failure, any allocated
+  /// buffers are freed and the state of 'range' is unmodified so that allocation can be
+  /// retried.  Setting 'max_bytes' to 3 * max_buffer_size() will typically maximize I/O
+  /// throughput. See Buffer management" section of the class comment for explanation.
+  /// TODO: error handling contract will change with reservations. The caller needs to
+  /// to guarantee that there is sufficient reservation.
+  Status AllocateBuffersForRange(RequestContext* reader, ScanRange* range,
+      int64_t max_bytes);
 
   /// Determine which disk queue this file should be assigned to.  Returns an index into
   /// disk_queues_.  The disk_id is the volume ID for the local disk that holds the
@@ -267,8 +319,8 @@ class DiskIoMgr : public CacheLineAligned {
   /// co-located with the datanode for this file.
   int AssignQueue(const char* file, int disk_id, bool expected_local);
 
-  /// Returns the maximum read buffer size
-  int64_t max_read_buffer_size() const { return max_buffer_size_; }
+  int64_t min_buffer_size() const { return min_buffer_size_; }
+  int64_t max_buffer_size() const { return max_buffer_size_; }
 
   /// Returns the total number of disk queues (both local and remote).
   int num_total_disks() const { return disk_queues_.size(); }
@@ -318,25 +370,6 @@ class DiskIoMgr : public CacheLineAligned {
   Status ReopenCachedHdfsFileHandle(const hdfsFS& fs, std::string* fname, int64_t mtime,
       CachedHdfsFileHandle** fid);
 
-  /// The maximum number of ready buffers that can be queued in a scan range. Having two
-  /// queued buffers (plus the buffer that is returned to the client) gives good
-  /// performance in most scenarios:
-  /// 1. If the consumer is consuming data faster than we can read from disk, then the
-  ///    queue will be empty most of the time because the buffer will be immediately
-  ///    pulled off the queue as soon as it is added. There will always be an I/O request
-  ///    in the disk queue to maximize I/O throughput, which is the bottleneck in this
-  ///    case.
-  /// 2. If we can read from disk faster than the consumer is consuming data, the queue
-  ///    will fill up and there will always be a buffer available for the consumer to
-  ///    read, so the consumer will not block and we maximize consumer throughput, which
-  ///    is the bottleneck in this case.
-  /// 3. If the consumer is consuming data at approximately the same rate as we are
-  ///    reading from disk, then the steady state is that the consumer is processing one
-  ///    buffer and one buffer is in the disk queue. The additional buffer can absorb
-  ///    bursts where the producer runs faster than the consumer or the consumer runs
-  ///    faster than the producer without blocking either the producer or consumer.
-  static const int SCAN_RANGE_READY_BUFFER_LIMIT = 2;
-
   /// "Disk" queue offsets for remote accesses.  Offset 0 corresponds to
   /// disk ID (i.e. disk_queue_ index) of num_local_disks().
   enum {
@@ -354,6 +387,7 @@ class DiskIoMgr : public CacheLineAligned {
   struct DiskQueue;
 
   friend class DiskIoMgrTest_Buffers_Test;
+  friend class DiskIoMgrTest_BufferSizeSelection_Test;
   friend class DiskIoMgrTest_VerifyNumThreadsParameter_Test;
 
   /// Number of worker(read) threads per rotational disk. Also the max depth of queued
@@ -442,17 +476,14 @@ class DiskIoMgr : public CacheLineAligned {
   /// Does not open or close the file that is written.
   Status WriteRangeHelper(FILE* file_handle, WriteRange* write_range) WARN_UNUSED_RESULT;
 
-  /// Reads the specified scan range and calls HandleReadFinished() when done.
+  /// Reads the specified scan range and calls HandleReadFinished() when done. If no
+  /// buffer is available to read the range's data into, the read cannot proceed, the
+  /// range becomes blocked and this function returns without doing I/O.
   void ReadRange(DiskQueue* disk_queue, RequestContext* reader, ScanRange* range);
 
-  /// Try to allocate the next buffer for the scan range, returning the new buffer
-  /// if successful. If 'reader' is cancelled, cancels the range and returns nullptr.
-  /// If there is memory pressure and buffers are already queued, adds the range
-  /// to the blocked ranges and returns nullptr. If buffers are not queued and no more
-  /// buffers can be allocated, cancels the range with a MEM_LIMIT_EXCEEDED error and
-  /// returns nullptr.
-  std::unique_ptr<BufferDescriptor> TryAllocateNextBufferForRange(DiskQueue* disk_queue,
-      RequestContext* reader, ScanRange* range, int64_t buffer_size);
+  /// Helper for AllocateBuffersForRange() to compute the buffer sizes for a scan range
+  /// with length 'scan_range_len', given that 'max_bytes' of memory should be allocated.
+  std::vector<int64_t> ChooseBufferSizes(int64_t scan_range_len, int64_t max_bytes);
 };
 }
 }


[06/15] impala git commit: IMPALA-4835: Part 3: switch I/O buffers to buffer pool

Posted by ta...@apache.org.
IMPALA-4835: Part 3: switch I/O buffers to buffer pool

This is the final patch to switch the Disk I/O manager to allocate all
buffer from the buffer pool and to reserve the buffers required for
a query upfront.

* The planner reserves enough memory to run a single scanner per
  scan node.
* The multi-threaded scan node must increase reservation before
  spinning up more threads.
* The scanner implementations must be careful to stay within their
  assigned reservation.

The row-oriented scanners were most straightforward, since they only
have a single scan range active at a time. A single I/O buffer is
sufficient to scan the whole file but more I/O buffers can improve I/O
throughput.

Parquet is more complex because it issues a scan range per column and
the sizes of the columns on disk are not known during planning. To
deal with this, the reservation in the frontend is based on a
heuristic involving the file size and # columns. The Parquet scanner
can then divvy up reservation to columns based on the size of column
data on disk.

I adjusted how the 'mem_limit' is divided between buffer pool and non
buffer pool memory for low mem_limits to account for the increase in
buffer pool memory.

Testing:
* Added more planner tests to cover reservation calcs for scan node.
* Test scanners for all file formats with the reservation denial debug
  action, to test behaviour when the scanners hit reservation limits.
* Updated memory and buffer pool limits for tests.
* Added unit tests for dividing reservation between columns in parquet,
  since the algorithm is non-trivial.

Perf:
I ran TPC-H and targeted perf locally comparing with master. Both
showed small improvements of a few percent and no regressions of
note. Cluster perf tests showed no significant change.

Conflicts:
	be/src/exec/parquet-column-readers.cc
	testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
	testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test

Other backport notes:
File sizes of functional_parquet.alltypes are slightly less than 16kb on
2.x but slightly more than 16kb on master because sorts are not clustered
by default. This changes resource requirements in some plans.

Change-Id: Ic09c6196b31e55b301df45cc56d0b72cfece6786
Reviewed-on: http://gerrit.cloudera.org:8080/9417
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/8c922a6e
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/8c922a6e
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/8c922a6e

Branch: refs/heads/2.x
Commit: 8c922a6ef675181c8457bf712a0f2146ab2fefd2
Parents: 0b6fab7
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Fri Jan 5 16:47:03 2018 -0800
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Fri Feb 23 22:51:01 2018 +0000

----------------------------------------------------------------------
 be/src/exec/CMakeLists.txt                      |    1 +
 be/src/exec/hdfs-parquet-scanner-test.cc        |   96 +
 be/src/exec/hdfs-parquet-scanner.cc             |  218 ++-
 be/src/exec/hdfs-parquet-scanner.h              |   38 +-
 be/src/exec/hdfs-scan-node-base.cc              |   27 +-
 be/src/exec/hdfs-scan-node-base.h               |    3 +
 be/src/exec/hdfs-scan-node-mt.cc                |   11 +-
 be/src/exec/hdfs-scan-node.cc                   |  143 +-
 be/src/exec/hdfs-scan-node.h                    |   66 +-
 be/src/exec/parquet-column-readers.cc           |  109 +-
 be/src/exec/parquet-column-readers.h            |   88 +-
 be/src/exec/scanner-context.cc                  |   28 +-
 be/src/exec/scanner-context.h                   |   54 +-
 .../bufferpool/reservation-tracker-test.cc      |    8 +-
 be/src/runtime/bufferpool/reservation-util.cc   |    2 +-
 be/src/runtime/io/disk-io-mgr-stress-test.cc    |   38 +-
 be/src/runtime/io/disk-io-mgr-stress.cc         |   69 +-
 be/src/runtime/io/disk-io-mgr-stress.h          |   26 +-
 be/src/runtime/io/disk-io-mgr-test.cc           |  542 +++---
 be/src/runtime/io/disk-io-mgr.cc                |   34 +-
 be/src/runtime/io/disk-io-mgr.h                 |   35 +-
 be/src/runtime/io/request-context.cc            |   29 +-
 be/src/runtime/io/request-context.h             |   14 +-
 be/src/runtime/io/request-ranges.h              |   15 +-
 be/src/runtime/io/scan-range.cc                 |    6 +-
 be/src/runtime/tmp-file-mgr.cc                  |    2 +-
 common/thrift/PlanNodes.thrift                  |    3 +
 .../apache/impala/analysis/SlotDescriptor.java  |   19 +
 .../org/apache/impala/analysis/SlotRef.java     |   20 -
 .../org/apache/impala/planner/HdfsScanNode.java |  167 +-
 .../java/org/apache/impala/util/BitUtil.java    |    6 +
 .../org/apache/impala/util/BitUtilTest.java     |    6 +
 .../queries/PlannerTest/constant-folding.test   |   42 +-
 .../queries/PlannerTest/disable-codegen.test    |   24 +-
 .../PlannerTest/fk-pk-join-detection.test       |   78 +-
 .../queries/PlannerTest/max-row-size.test       |   80 +-
 .../PlannerTest/min-max-runtime-filters.test    |    6 +-
 .../queries/PlannerTest/mt-dop-validation.test  |   40 +-
 .../queries/PlannerTest/parquet-filtering.test  |   42 +-
 .../queries/PlannerTest/partition-pruning.test  |    4 +-
 .../PlannerTest/resource-requirements.test      | 1814 ++++++++++++++----
 .../PlannerTest/sort-expr-materialization.test  |   32 +-
 .../PlannerTest/spillable-buffer-sizing.test    |  192 +-
 .../queries/PlannerTest/tablesample.test        |   44 +-
 .../queries/PlannerTest/union.test              |    8 +-
 .../admission-reject-min-reservation.test       |   12 +-
 .../queries/QueryTest/analytic-fns.test         |    5 +-
 .../queries/QueryTest/codegen-mem-limit.test    |    5 +-
 .../QueryTest/disk-spill-encryption.test        |    2 +-
 .../queries/QueryTest/explain-level0.test       |    4 +-
 .../queries/QueryTest/explain-level1.test       |    4 +-
 .../queries/QueryTest/explain-level2.test       |   14 +-
 .../queries/QueryTest/explain-level3.test       |   14 +-
 .../queries/QueryTest/nested-types-tpch.test    |    6 +-
 .../queries/QueryTest/runtime_row_filters.test  |   11 +-
 .../queries/QueryTest/scanners.test             |    7 +
 .../functional-query/queries/QueryTest/set.test |    2 +-
 .../queries/QueryTest/spilling-aggs.test        |   19 +-
 .../spilling-naaj-no-deny-reservation.test      |    7 +-
 .../queries/QueryTest/spilling-naaj.test        |    8 +-
 .../QueryTest/spilling-no-debug-action.test     |   66 +
 .../QueryTest/spilling-sorts-exhaustive.test    |   10 +-
 .../queries/QueryTest/spilling.test             |   76 +-
 .../queries/QueryTest/stats-extrapolation.test  |   52 +-
 .../tpch/queries/sort-reservation-usage.test    |    9 +-
 tests/common/test_dimensions.py                 |   16 +-
 tests/custom_cluster/test_scratch_disk.py       |    2 +-
 tests/query_test/test_mem_usage_scaling.py      |   11 +-
 tests/query_test/test_query_mem_limit.py        |    4 +-
 tests/query_test/test_scanners.py               |   25 +-
 tests/query_test/test_scanners_fuzz.py          |    9 +-
 tests/query_test/test_sort.py                   |   16 +-
 tests/query_test/test_spilling.py               |    5 +
 73 files changed, 3204 insertions(+), 1546 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index aab1383..c1f91d6 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -108,3 +108,4 @@ ADD_BE_TEST(parquet-version-test)
 ADD_BE_TEST(row-batch-list-test)
 ADD_BE_TEST(incr-stats-util-test)
 ADD_BE_TEST(hdfs-avro-scanner-test)
+ADD_BE_TEST(hdfs-parquet-scanner-test)

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/hdfs-parquet-scanner-test.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner-test.cc b/be/src/exec/hdfs-parquet-scanner-test.cc
new file mode 100644
index 0000000..cbc6e76
--- /dev/null
+++ b/be/src/exec/hdfs-parquet-scanner-test.cc
@@ -0,0 +1,96 @@
+// 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/hdfs-parquet-scanner.h"
+#include "testutil/gtest-util.h"
+
+#include "common/names.h"
+
+static const int64_t MIN_BUFFER_SIZE = 64 * 1024;
+static const int64_t MAX_BUFFER_SIZE = 8 * 1024 * 1024;
+
+namespace impala {
+
+class HdfsParquetScannerTest : public testing::Test {
+ protected:
+  void TestDivideReservation(const vector<int64_t>& col_range_lengths,
+      int64_t total_col_reservation, const vector<int64_t>& expected_reservations);
+};
+
+/// Test that DivideReservationBetweenColumns() returns 'expected_reservations' for
+/// inputs 'col_range_lengths' and 'total_col_reservation'.
+void HdfsParquetScannerTest::TestDivideReservation(const vector<int64_t>& col_range_lengths,
+      int64_t total_col_reservation, const vector<int64_t>& expected_reservations) {
+  vector<pair<int, int64_t>> reservations =
+      HdfsParquetScanner::DivideReservationBetweenColumnsHelper(
+      MIN_BUFFER_SIZE, MAX_BUFFER_SIZE, col_range_lengths, total_col_reservation);
+  for (int i = 0; i < reservations.size(); ++i) {
+    LOG(INFO) << i << " " << reservations[i].first << " " << reservations[i].second;
+  }
+  EXPECT_EQ(reservations.size(), expected_reservations.size());
+  vector<bool> present(expected_reservations.size(), false);
+  for (auto& reservation: reservations) {
+    // Ensure that each appears exactly once.
+    EXPECT_FALSE(present[reservation.first]);
+    present[reservation.first] = true;
+    EXPECT_EQ(expected_reservations[reservation.first], reservation.second)
+        << reservation.first;
+  }
+}
+
+TEST_F(HdfsParquetScannerTest, DivideReservation) {
+  // Test a long scan ranges with lots of memory - should allocate 3 max-size
+  // buffers per range.
+  TestDivideReservation({100 * 1024 * 1024}, 50 * 1024 * 1024, {3 * MAX_BUFFER_SIZE});
+  TestDivideReservation({100 * 1024 * 1024, 50 * 1024 * 1024}, 100 * 1024 * 1024,
+        {3 * MAX_BUFFER_SIZE, 3 * MAX_BUFFER_SIZE});
+
+  // Long scan ranges, not enough memory for 3 buffers each. Should only allocate
+  // max-sized buffers, preferring the longer scan range.
+  TestDivideReservation({50 * 1024 * 1024, 100 * 1024 * 1024}, 5 * MAX_BUFFER_SIZE,
+        {2 * MAX_BUFFER_SIZE, 3 * MAX_BUFFER_SIZE});
+  TestDivideReservation({50 * 1024 * 1024, 100 * 1024 * 1024},
+        5 * MAX_BUFFER_SIZE + MIN_BUFFER_SIZE,
+        {2 * MAX_BUFFER_SIZE, 3 * MAX_BUFFER_SIZE});
+  TestDivideReservation({50 * 1024 * 1024, 100 * 1024 * 1024}, 6 * MAX_BUFFER_SIZE - 1,
+        {2 * MAX_BUFFER_SIZE, 3 * MAX_BUFFER_SIZE});
+
+  // Test a short range with lots of memory - should round up buffer size.
+  TestDivideReservation({100 * 1024}, 50 * 1024 * 1024, {128 * 1024});
+
+  // Test a range << MIN_BUFFER_SIZE - should round up to buffer size.
+  TestDivideReservation({13}, 50 * 1024 * 1024, {MIN_BUFFER_SIZE});
+
+  // Test long ranges with limited memory.
+  TestDivideReservation({100 * 1024 * 1024}, 100 * 1024, {MIN_BUFFER_SIZE});
+  TestDivideReservation({100 * 1024 * 1024}, MIN_BUFFER_SIZE, {MIN_BUFFER_SIZE});
+  TestDivideReservation({100 * 1024 * 1024}, 2 * MIN_BUFFER_SIZE, {2 * MIN_BUFFER_SIZE});
+  TestDivideReservation({100 * 1024 * 1024}, MAX_BUFFER_SIZE - 1, {MAX_BUFFER_SIZE / 2});
+  TestDivideReservation({100 * 1024 * 1024, 1024 * 1024, MIN_BUFFER_SIZE},
+      3 * MIN_BUFFER_SIZE, {MIN_BUFFER_SIZE, MIN_BUFFER_SIZE, MIN_BUFFER_SIZE});
+
+  // Test a mix of scan range lengths larger than and smaller than the max I/O buffer
+  // size. Long ranges get allocated most memory.
+  TestDivideReservation(
+      {15145047, 5019635, 5019263, 15145047, 15145047, 5019635, 5019263, 317304},
+      25165824,
+      {8388608, 2097152, 524288, 8388608, 4194304, 1048576, 262144, 262144});
+}
+
+}
+
+IMPALA_TEST_MAIN();

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index 0188f08..51a39be 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -17,6 +17,7 @@
 
 #include "exec/hdfs-parquet-scanner.h"
 
+#include <algorithm>
 #include <queue>
 
 #include <gutil/strings/substitute.h>
@@ -27,6 +28,7 @@
 #include "exec/parquet-column-stats.h"
 #include "exec/scanner-context.inline.h"
 #include "runtime/collection-value-builder.h"
+#include "runtime/exec-env.h"
 #include "runtime/io/disk-io-mgr.h"
 #include "runtime/runtime-state.h"
 #include "runtime/runtime-filter.inline.h"
@@ -35,6 +37,7 @@
 #include "common/names.h"
 
 using std::move;
+using std::sort;
 using namespace impala;
 using namespace impala::io;
 
@@ -47,10 +50,6 @@ constexpr int BATCHES_PER_FILTER_SELECTIVITY_CHECK = 16;
 static_assert(BitUtil::IsPowerOf2(BATCHES_PER_FILTER_SELECTIVITY_CHECK),
     "BATCHES_PER_FILTER_SELECTIVITY_CHECK must be a power of two");
 
-// Max dictionary page header size in bytes. This is an estimate and only needs to be an
-// upper bound.
-const int MAX_DICT_HEADER_SIZE = 100;
-
 // Max entries in the dictionary before switching to PLAIN encoding. If a dictionary
 // has fewer entries, then the entire column is dictionary encoded. This threshold
 // is guaranteed to be true for Impala versions 2.9 or below.
@@ -99,7 +98,7 @@ Status HdfsParquetScanner::IssueInitialRanges(HdfsScanNodeBase* scan_node,
             static_cast<ScanRangeMetadata*>(split->meta_data());
         // Each split is processed by first issuing a scan range for the file footer, which
         // is done here, followed by scan ranges for the columns of each row group within
-        // the actual split (in InitColumns()). The original split is stored in the
+        // the actual split (see InitScalarColumns()). The original split is stored in the
         // metadata associated with the footer range.
         ScanRange* footer_range;
         if (footer_split != nullptr) {
@@ -229,9 +228,14 @@ Status HdfsParquetScanner::Open(ScannerContext* context) {
 
   // Release I/O buffers immediately to make sure they are cleaned up
   // in case we return a non-OK status anywhere below.
+  int64_t stream_reservation = stream_->reservation();
+  stream_ = nullptr;
   context_->ReleaseCompletedResources(true);
   context_->ClearStreams();
   RETURN_IF_ERROR(footer_status);
+  // The scanner-wide stream was used only to read the file footer.  Each column has added
+  // its own stream. We can use the reservation from 'stream_' for the columns now.
+  total_col_reservation_ = stream_reservation;
 
   // Parse the file schema into an internal representation for schema resolution.
   schema_resolver_.reset(new ParquetSchemaResolver(*scan_node_->hdfs_table(),
@@ -248,10 +252,6 @@ Status HdfsParquetScanner::Open(ScannerContext* context) {
   template_tuple_ = template_tuple_map_[scan_node_->tuple_desc()];
 
   RETURN_IF_ERROR(InitDictFilterStructures());
-
-  // The scanner-wide stream was used only to read the file footer.  Each column has added
-  // its own stream.
-  stream_ = nullptr;
   return Status::OK();
 }
 
@@ -675,15 +675,13 @@ Status HdfsParquetScanner::NextRowGroup() {
     }
 
     InitCollectionColumns();
+    RETURN_IF_ERROR(InitScalarColumns());
 
-    // Prepare dictionary filtering columns for first read
-    // This must be done before dictionary filtering, because this code initializes
-    // the column offsets and streams needed to read the dictionaries.
-    // TODO: Restructure the code so that the dictionary can be read without the rest
-    // of the column.
-    RETURN_IF_ERROR(InitScalarColumns(row_group_idx_, dict_filterable_readers_));
+    // Start scanning dictionary filtering column readers, so we can read the dictionary
+    // pages in EvalDictionaryFilters().
+    RETURN_IF_ERROR(BaseScalarColumnReader::StartScans(dict_filterable_readers_));
 
-    // InitColumns() may have allocated resources to scan columns. If we skip this row
+    // StartScans() may have allocated resources to scan columns. If we skip this row
     // group below, we must call ReleaseSkippedRowGroupResources() before continuing.
 
     // If there is a dictionary-encoded column where every value is eliminated
@@ -704,10 +702,10 @@ Status HdfsParquetScanner::NextRowGroup() {
     }
 
     // At this point, the row group has passed any filtering criteria
-    // Prepare non-dictionary filtering column readers for first read and
-    // initialize their dictionaries.
-    RETURN_IF_ERROR(InitScalarColumns(row_group_idx_, non_dict_filterable_readers_));
-    status = InitDictionaries(non_dict_filterable_readers_);
+    // Start scanning non-dictionary filtering column readers and initialize their
+    // dictionaries.
+    RETURN_IF_ERROR(BaseScalarColumnReader::StartScans(non_dict_filterable_readers_));
+    status = BaseScalarColumnReader::InitDictionaries(non_dict_filterable_readers_);
     if (!status.ok()) {
       // Either return an error or skip this row group if it is ok to ignore errors
       RETURN_IF_ERROR(state_->LogOrReturnError(status.msg()));
@@ -742,7 +740,6 @@ Status HdfsParquetScanner::NextRowGroup() {
       break;
     }
   }
-
   DCHECK(parse_status_.ok());
   return Status::OK();
 }
@@ -800,6 +797,7 @@ void HdfsParquetScanner::PartitionReaders(
     } else {
       BaseScalarColumnReader* scalar_reader =
           static_cast<BaseScalarColumnReader*>(reader);
+      scalar_readers_.push_back(scalar_reader);
       if (can_eval_dict_filters && IsDictFilterable(scalar_reader)) {
         dict_filterable_readers_.push_back(scalar_reader);
       } else {
@@ -991,7 +989,7 @@ Status HdfsParquetScanner::EvalDictionaryFilters(const parquet::RowGroup& row_gr
 
   // Any columns that were not 100% dictionary encoded need to initialize
   // their dictionaries here.
-  RETURN_IF_ERROR(InitDictionaries(deferred_dict_init_list));
+  RETURN_IF_ERROR(BaseScalarColumnReader::InitDictionaries(deferred_dict_init_list));
 
   return Status::OK();
 }
@@ -1648,23 +1646,16 @@ void HdfsParquetScanner::InitCollectionColumns() {
   }
 }
 
-Status HdfsParquetScanner::InitScalarColumns(
-    int row_group_idx, const vector<BaseScalarColumnReader*>& column_readers) {
+Status HdfsParquetScanner::InitScalarColumns() {
   int64_t partition_id = context_->partition_descriptor()->id();
   const HdfsFileDesc* file_desc = scan_node_->GetFileDesc(partition_id, filename());
   DCHECK(file_desc != nullptr);
-  parquet::RowGroup& row_group = file_metadata_.row_groups[row_group_idx];
+  parquet::RowGroup& row_group = file_metadata_.row_groups[row_group_idx_];
 
-  // All the scan ranges (one for each column).
-  vector<ScanRange*> col_ranges;
   // Used to validate that the number of values in each reader in column_readers_ at the
   // same SchemaElement is the same.
   unordered_map<const parquet::SchemaElement*, int> num_values_map;
-  // Used to validate we issued the right number of scan ranges
-  int num_scalar_readers = 0;
-
-  for (BaseScalarColumnReader* scalar_reader: column_readers) {
-    ++num_scalar_readers;
+  for (BaseScalarColumnReader* scalar_reader : scalar_readers_) {
     const parquet::ColumnChunk& col_chunk = row_group.columns[scalar_reader->col_idx()];
     auto num_values_it = num_values_map.find(&scalar_reader->schema_element());
     int num_values = -1;
@@ -1673,84 +1664,115 @@ Status HdfsParquetScanner::InitScalarColumns(
     } else {
       num_values_map[&scalar_reader->schema_element()] = col_chunk.meta_data.num_values;
     }
-    int64_t col_start = col_chunk.meta_data.data_page_offset;
-
     if (num_values != -1 && col_chunk.meta_data.num_values != num_values) {
       // TODO: improve this error message by saying which columns are different,
       // and also specify column in other error messages as appropriate
       return Status(TErrorCode::PARQUET_NUM_COL_VALS_ERROR, scalar_reader->col_idx(),
           col_chunk.meta_data.num_values, num_values, filename());
     }
+    RETURN_IF_ERROR(scalar_reader->Reset(*file_desc, col_chunk, row_group_idx_));
+  }
+  RETURN_IF_ERROR(
+      DivideReservationBetweenColumns(scalar_readers_, total_col_reservation_));
+  return Status::OK();
+}
 
-    RETURN_IF_ERROR(ParquetMetadataUtils::ValidateRowGroupColumn(file_metadata_,
-        filename(), row_group_idx, scalar_reader->col_idx(),
-        scalar_reader->schema_element(), state_));
-
-    if (col_chunk.meta_data.__isset.dictionary_page_offset) {
-      // Already validated in ValidateColumnOffsets()
-      DCHECK_LT(col_chunk.meta_data.dictionary_page_offset, col_start);
-      col_start = col_chunk.meta_data.dictionary_page_offset;
-    }
-    int64_t col_len = col_chunk.meta_data.total_compressed_size;
-    if (col_len <= 0) {
-      return Status(Substitute("File '$0' contains invalid column chunk size: $1",
-          filename(), col_len));
-    }
-    int64_t col_end = col_start + col_len;
-
-    // Already validated in ValidateColumnOffsets()
-    DCHECK_GT(col_end, 0);
-    DCHECK_LT(col_end, file_desc->file_length);
-    if (file_version_.application == "parquet-mr" && file_version_.VersionLt(1, 2, 9)) {
-      // The Parquet MR writer had a bug in 1.2.8 and below where it didn't include the
-      // dictionary page header size in total_compressed_size and total_uncompressed_size
-      // (see IMPALA-694). We pad col_len to compensate.
-      int64_t bytes_remaining = file_desc->file_length - col_end;
-      int64_t pad = min<int64_t>(MAX_DICT_HEADER_SIZE, bytes_remaining);
-      col_len += pad;
-    }
-
-    // TODO: this will need to change when we have co-located files and the columns
-    // are different files.
-    if (!col_chunk.file_path.empty() && col_chunk.file_path != filename()) {
-      return Status(Substitute("Expected parquet column file path '$0' to match "
-          "filename '$1'", col_chunk.file_path, filename()));
-    }
-
-    const ScanRange* split_range =
-        static_cast<ScanRangeMetadata*>(metadata_range_->meta_data())->original_split;
-
-    // Determine if the column is completely contained within a local split.
-    bool col_range_local = split_range->expected_local()
-        && col_start >= split_range->offset()
-        && col_end <= split_range->offset() + split_range->len();
-    ScanRange* col_range = scan_node_->AllocateScanRange(metadata_range_->fs(),
-        filename(), col_len, col_start, partition_id, split_range->disk_id(),
-        col_range_local,
-        BufferOpts(split_range->try_cache(), file_desc->mtime));
-    col_ranges.push_back(col_range);
-
-    // Get the stream that will be used for this column
-    ScannerContext::Stream* stream = context_->AddStream(col_range);
-    DCHECK(stream != nullptr);
+Status HdfsParquetScanner::DivideReservationBetweenColumns(
+    const vector<BaseScalarColumnReader*>& column_readers,
+    int64_t reservation_to_distribute) {
+  DiskIoMgr* io_mgr = ExecEnv::GetInstance()->disk_io_mgr();
+  const int64_t min_buffer_size = io_mgr->min_buffer_size();
+  const int64_t max_buffer_size = io_mgr->max_buffer_size();
+  // The HdfsScanNode reservation calculation in the planner ensures that we have
+  // reservation for at least one buffer per column.
+  if (reservation_to_distribute < min_buffer_size * column_readers.size()) {
+    return Status(TErrorCode::INTERNAL_ERROR,
+        Substitute("Not enough reservation in Parquet scanner for file '$0'. Need at "
+                   "least $1 bytes per column for $2 columns but had $3 bytes",
+            filename(), min_buffer_size, column_readers.size(),
+            reservation_to_distribute));
+  }
 
-    RETURN_IF_ERROR(scalar_reader->Reset(&col_chunk.meta_data, stream));
+  vector<int64_t> col_range_lengths(column_readers.size());
+  for (int i = 0; i < column_readers.size(); ++i) {
+    col_range_lengths[i] = column_readers[i]->scan_range()->len();
   }
-  DCHECK_EQ(col_ranges.size(), num_scalar_readers);
+  vector<pair<int, int64_t>> tmp_reservations = DivideReservationBetweenColumnsHelper(
+      min_buffer_size, max_buffer_size, col_range_lengths, reservation_to_distribute);
+  for (auto& tmp_reservation : tmp_reservations) {
+    column_readers[tmp_reservation.first]->set_io_reservation(tmp_reservation.second);
+  }
+  return Status::OK();
+}
 
-  DiskIoMgr* io_mgr = scan_node_->runtime_state()->io_mgr();
-  // Issue all the column chunks to the IoMgr. We scan through all columns at the same
-  // time so need to read from all of them concurrently.
-  for (ScanRange* col_range : col_ranges) {
-    bool needs_buffers;
-    RETURN_IF_ERROR(io_mgr->StartScanRange(
-        scan_node_->reader_context(), col_range, &needs_buffers));
-    if (needs_buffers) {
-      RETURN_IF_ERROR(io_mgr->AllocateBuffersForRange(
-          scan_node_->reader_context(), col_range, 3 * io_mgr->max_buffer_size()));
+vector<pair<int, int64_t>> HdfsParquetScanner::DivideReservationBetweenColumnsHelper(
+    int64_t min_buffer_size, int64_t max_buffer_size,
+    const vector<int64_t>& col_range_lengths, int64_t reservation_to_distribute) {
+  // Pair of (column index, reservation allocated).
+  vector<pair<int, int64_t>> tmp_reservations;
+  for (int i = 0; i < col_range_lengths.size(); ++i) tmp_reservations.emplace_back(i, 0);
+
+  // Sort in descending order of length, breaking ties by index so that larger columns
+  // get allocated reservation first. It is common to have dramatically different column
+  // sizes in a single file because of different value sizes and compressibility. E.g.
+  // consider a large STRING "comment" field versus a highly compressible
+  // dictionary-encoded column with only a few distinct values. We want to give max-sized
+  // buffers to large columns first to maximize the size of I/Os that we do while reading
+  // this row group.
+  sort(tmp_reservations.begin(), tmp_reservations.end(),
+      [&col_range_lengths](const pair<int, int64_t>& left, const pair<int, int64_t>& right) {
+        int64_t left_len = col_range_lengths[left.first];
+        int64_t right_len = col_range_lengths[right.first];
+        return left_len != right_len ? left_len > right_len : left.first < right.first;
+      });
+
+  // Set aside the minimum reservation per column.
+  reservation_to_distribute -= min_buffer_size * col_range_lengths.size();
+
+  // Allocate reservations to columns by repeatedly allocating either a max-sized buffer
+  // or a large enough buffer to fit the remaining data for each column. Do this
+  // round-robin up to the ideal number of I/O buffers.
+  for (int i = 0; i < DiskIoMgr::IDEAL_MAX_SIZED_BUFFERS_PER_SCAN_RANGE; ++i) {
+    for (auto& tmp_reservation : tmp_reservations) {
+      // Add back the reservation we set aside above.
+      if (i == 0) reservation_to_distribute += min_buffer_size;
+
+      int64_t bytes_left_in_range =
+          col_range_lengths[tmp_reservation.first] - tmp_reservation.second;
+      int64_t bytes_to_add;
+      if (bytes_left_in_range >= max_buffer_size) {
+        if (reservation_to_distribute >= max_buffer_size) {
+          bytes_to_add = max_buffer_size;
+        } else if (i == 0) {
+          DCHECK_EQ(0, tmp_reservation.second);
+          // Ensure this range gets at least one buffer on the first iteration.
+          bytes_to_add = BitUtil::RoundDownToPowerOfTwo(reservation_to_distribute);
+        } else {
+          DCHECK_GT(tmp_reservation.second, 0);
+          // We need to read more than the max buffer size, but can't allocate a
+          // max-sized buffer. Stop adding buffers to this column: we prefer to use
+          // the existing max-sized buffers without small buffers mixed in so that
+          // we will alway do max-sized I/Os, which make efficient use of I/O devices.
+          bytes_to_add = 0;
+        }
+      } else if (bytes_left_in_range > 0 &&
+          reservation_to_distribute >= min_buffer_size) {
+        // Choose a buffer size that will fit the rest of the bytes left in the range.
+        bytes_to_add = max(min_buffer_size, BitUtil::RoundUpToPowerOfTwo(bytes_left_in_range));
+        // But don't add more reservation than is available.
+        bytes_to_add =
+            min(bytes_to_add, BitUtil::RoundDownToPowerOfTwo(reservation_to_distribute));
+      } else {
+        bytes_to_add = 0;
+      }
+      DCHECK(bytes_to_add == 0 || bytes_to_add >= min_buffer_size) << bytes_to_add;
+      reservation_to_distribute -= bytes_to_add;
+      tmp_reservation.second += bytes_to_add;
+      DCHECK_GE(reservation_to_distribute, 0);
+      DCHECK_GT(tmp_reservation.second, 0);
     }
   }
-  return Status::OK();
+  return tmp_reservations;
 }
 
 Status HdfsParquetScanner::InitDictionaries(

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/hdfs-parquet-scanner.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.h b/be/src/exec/hdfs-parquet-scanner.h
index 1fc3239..92f2550 100644
--- a/be/src/exec/hdfs-parquet-scanner.h
+++ b/be/src/exec/hdfs-parquet-scanner.h
@@ -34,7 +34,7 @@ class CollectionValueBuilder;
 struct HdfsFileDesc;
 
 /// Internal schema representation and resolution.
-class SchemaNode;
+struct SchemaNode;
 
 /// Class that implements Parquet definition and repetition level decoding.
 class ParquetLevelDecoder;
@@ -361,6 +361,7 @@ class HdfsParquetScanner : public HdfsScanner {
   template<typename InternalType, parquet::Type::type PARQUET_TYPE, bool MATERIALIZED>
   friend class ScalarColumnReader;
   friend class BoolColumnReader;
+  friend class HdfsParquetScannerTest;
 
   /// Size of the file footer.  This is a guess.  If this value is too little, we will
   /// need to issue another read.
@@ -429,7 +430,7 @@ class HdfsParquetScanner : public HdfsScanner {
   /// Number of scratch batches processed so far.
   int64_t row_batches_produced_;
 
-  /// Column reader for each materialized columns for this file.
+  /// Column reader for each top-level materialized slot in the output tuple.
   std::vector<ParquetColumnReader*> column_readers_;
 
   /// Column readers will write slot values into this scratch batch for
@@ -445,6 +446,9 @@ class HdfsParquetScanner : public HdfsScanner {
   /// Scan range for the metadata.
   const io::ScanRange* metadata_range_;
 
+  /// Reservation available for scanning columns, in bytes.
+  int64_t total_col_reservation_ = 0;
+
   /// Pool to copy dictionary page buffer into. This pool is shared across all the
   /// pages in a column chunk.
   boost::scoped_ptr<MemPool> dictionary_pool_;
@@ -461,6 +465,9 @@ class HdfsParquetScanner : public HdfsScanner {
   /// or nested within a collection.
   std::vector<BaseScalarColumnReader*> non_dict_filterable_readers_;
 
+  /// Flattened list of all scalar column readers in column_readers_.
+  std::vector<BaseScalarColumnReader*> scalar_readers_;
+
   /// Flattened collection column readers that point to readers in column_readers_.
   std::vector<CollectionColumnReader*> collection_readers_;
 
@@ -627,12 +634,24 @@ class HdfsParquetScanner : public HdfsScanner {
       WARN_UNUSED_RESULT;
 
   /// Walks file_metadata_ and initiates reading the materialized columns.  This
-  /// initializes 'column_readers' and issues the reads for the columns. 'column_readers'
-  /// includes a mix of scalar readers from multiple schema nodes (i.e., readers of
-  /// top-level scalar columns and readers of scalar columns within a collection node).
-  Status InitScalarColumns(
-      int row_group_idx, const std::vector<BaseScalarColumnReader*>& column_readers)
-      WARN_UNUSED_RESULT;
+  /// initializes 'scalar_readers_' and divides reservation between the columns but
+  /// does not start any scan ranges.
+  Status InitScalarColumns() WARN_UNUSED_RESULT;
+
+  /// Decides how to divide 'reservation_to_distribute' bytes of reservation between the
+  /// columns. Sets the reservation on each corresponding reader in 'column_readers'.
+  Status DivideReservationBetweenColumns(
+      const std::vector<BaseScalarColumnReader*>& column_readers,
+      int64_t reservation_to_distribute);
+
+  /// Helper for DivideReservationBetweenColumns. Implements the core algorithm for
+  /// dividing a reservation of 'reservation_to_distribute' bytes between columns with
+  /// scan range lengths 'col_range_lengths' given a min and max buffer size. Returns
+  /// a vector with an entry per column with the index into 'col_range_lengths' and the
+  /// amount of reservation in bytes to give to that column.
+  static std::vector<std::pair<int, int64_t>> DivideReservationBetweenColumnsHelper(
+      int64_t min_buffer_size, int64_t max_buffer_size,
+      const std::vector<int64_t>& col_range_lengths, int64_t reservation_to_distribute);
 
   /// Initializes the column readers in collection_readers_.
   void InitCollectionColumns();
@@ -668,7 +687,8 @@ class HdfsParquetScanner : public HdfsScanner {
   /// Partitions the readers into scalar and collection readers. The collection readers
   /// are flattened into collection_readers_. The scalar readers are partitioned into
   /// dict_filterable_readers_ and non_dict_filterable_readers_ depending on whether
-  /// dictionary filtering is enabled and the reader can be dictionary filtered.
+  /// dictionary filtering is enabled and the reader can be dictionary filtered. All
+  /// scalar readers are also flattened into scalar_readers_.
   void PartitionReaders(const vector<ParquetColumnReader*>& readers,
                         bool can_eval_dict_filters);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/hdfs-scan-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index 861d5dc..164197d 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -35,6 +35,7 @@
 #include "exprs/scalar-expr-evaluator.h"
 #include "exprs/scalar-expr.h"
 #include "runtime/descriptors.h"
+#include "runtime/exec-env.h"
 #include "runtime/hdfs-fs-cache.h"
 #include "runtime/io/disk-io-mgr.h"
 #include "runtime/io/request-context.h"
@@ -67,6 +68,7 @@ const int UNEXPECTED_REMOTE_BYTES_WARN_THRESHOLD = 64 * 1024 * 1024;
 HdfsScanNodeBase::HdfsScanNodeBase(ObjectPool* pool, const TPlanNode& tnode,
     const DescriptorTbl& descs)
     : ScanNode(pool, tnode, descs),
+      ideal_scan_range_reservation_(tnode.hdfs_scan_node.ideal_scan_range_reservation),
       min_max_tuple_id_(tnode.hdfs_scan_node.__isset.min_max_tuple_id ?
           tnode.hdfs_scan_node.min_max_tuple_id : -1),
       skip_header_line_count_(tnode.hdfs_scan_node.__isset.skip_header_line_count ?
@@ -83,6 +85,7 @@ HdfsScanNodeBase::HdfsScanNodeBase(ObjectPool* pool, const TPlanNode& tnode,
           &tnode.hdfs_scan_node.dictionary_filter_conjuncts : nullptr),
       disks_accessed_bitmap_(TUnit::UNIT, 0),
       active_hdfs_read_thread_counter_(TUnit::UNIT, 0) {
+  DCHECK_GE(ideal_scan_range_reservation_, resource_profile_.min_reservation);
 }
 
 HdfsScanNodeBase::~HdfsScanNodeBase() {
@@ -112,7 +115,6 @@ Status HdfsScanNodeBase::Init(const TPlanNode& tnode, RuntimeState* state) {
     RETURN_IF_ERROR(ScalarExpr::Create(tnode.hdfs_scan_node.min_max_conjuncts,
         *min_max_row_desc, state, &min_max_conjuncts_));
   }
-
   return Status::OK();
 }
 
@@ -246,6 +248,16 @@ Status HdfsScanNodeBase::Prepare(RuntimeState* state) {
   ImpaladMetrics::NUM_RANGES_PROCESSED->Increment(scan_range_params_->size());
   ImpaladMetrics::NUM_RANGES_MISSING_VOLUME_ID->Increment(num_ranges_missing_volume_id);
 
+  // Check if reservation was enough to allocate at least one buffer. The
+  // reservation calculation in HdfsScanNode.java should guarantee this.
+  // Hitting this error indicates a misconfiguration or bug.
+  int64_t min_buffer_size = ExecEnv::GetInstance()->disk_io_mgr()->min_buffer_size();
+  if (scan_range_params_->size() > 0
+      && resource_profile_.min_reservation < min_buffer_size) {
+    return Status(TErrorCode::INTERNAL_ERROR,
+      Substitute("HDFS scan min reservation $0 must be >= min buffer size $1",
+       resource_profile_.min_reservation, min_buffer_size));
+  }
   // Add per volume stats to the runtime profile
   PerVolumeStats per_volume_stats;
   stringstream str;
@@ -332,7 +344,18 @@ Status HdfsScanNodeBase::Open(RuntimeState* state) {
         partition_desc->partition_key_value_evals(), scan_node_pool_.get(), state);
   }
 
-  reader_context_ = runtime_state_->io_mgr()->RegisterContext(mem_tracker());
+  RETURN_IF_ERROR(ClaimBufferReservation(state));
+  // We got the minimum reservation. Now try to get ideal reservation.
+  if (resource_profile_.min_reservation != ideal_scan_range_reservation_) {
+    bool increased = buffer_pool_client_.IncreaseReservation(
+        ideal_scan_range_reservation_ - resource_profile_.min_reservation);
+    VLOG_FILE << "Increasing reservation from minimum "
+              << resource_profile_.min_reservation << "B to ideal "
+              << ideal_scan_range_reservation_ << "B "
+              << (increased ? "succeeded" : "failed");
+  }
+
+  reader_context_ = runtime_state_->io_mgr()->RegisterContext();
 
   // Initialize HdfsScanNode specific counters
   // TODO: Revisit counters and move the counters specific to multi-threaded scans

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/hdfs-scan-node-base.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.h b/be/src/exec/hdfs-scan-node-base.h
index 70fbac2..3a9c37f 100644
--- a/be/src/exec/hdfs-scan-node-base.h
+++ b/be/src/exec/hdfs-scan-node-base.h
@@ -322,6 +322,9 @@ class HdfsScanNodeBase : public ScanNode {
   friend class ScannerContext;
   friend class HdfsScanner;
 
+  /// Ideal reservation to process each input split, computed by the planner.
+  const int64_t ideal_scan_range_reservation_;
+
   /// Tuple id of the tuple used to evaluate conjuncts on parquet::Statistics.
   const int min_max_tuple_id_;
 

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/hdfs-scan-node-mt.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-mt.cc b/be/src/exec/hdfs-scan-node-mt.cc
index be75677..f4948d9 100644
--- a/be/src/exec/hdfs-scan-node-mt.cc
+++ b/be/src/exec/hdfs-scan-node-mt.cc
@@ -86,17 +86,18 @@ Status HdfsScanNodeMt::GetNext(RuntimeState* state, RowBatch* row_batch, bool* e
       StopAndFinalizeCounters();
       return Status::OK();
     }
+    int64_t scanner_reservation = buffer_pool_client_.GetReservation();
     if (needs_buffers) {
-      RETURN_IF_ERROR(io_mgr->AllocateBuffersForRange(reader_context_.get(), scan_range_,
-          3 * io_mgr->max_buffer_size()));
+      RETURN_IF_ERROR(io_mgr->AllocateBuffersForRange(reader_context_.get(),
+          &buffer_pool_client_, scan_range_, scanner_reservation));
     }
     ScanRangeMetadata* metadata =
         static_cast<ScanRangeMetadata*>(scan_range_->meta_data());
     int64_t partition_id = metadata->partition_id;
     HdfsPartitionDescriptor* partition = hdfs_table_->GetPartition(partition_id);
-    scanner_ctx_.reset(new ScannerContext(
-        runtime_state_, this, partition, scan_range_, filter_ctxs(),
-        expr_results_pool()));
+    scanner_ctx_.reset(new ScannerContext(runtime_state_, this, &buffer_pool_client_,
+        partition, filter_ctxs(), expr_results_pool()));
+    scanner_ctx_->AddStream(scan_range_, scanner_reservation);
     Status status = CreateAndOpenScanner(partition, scanner_ctx_.get(), &scanner_);
     if (!status.ok()) {
       DCHECK(scanner_ == NULL);

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/hdfs-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node.cc b/be/src/exec/hdfs-scan-node.cc
index f9d71e9..a95e47a 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -46,16 +46,6 @@ DECLARE_bool(skip_file_runtime_filtering);
 using namespace impala;
 using namespace impala::io;
 
-// Amount of memory that we approximate a scanner thread will use not including IoBuffers.
-// The memory used does not vary considerably between file formats (just a couple of MBs).
-// This value is conservative and taken from running against the tpch lineitem table.
-// TODO: revisit how we do this.
-const int SCANNER_THREAD_MEM_USAGE = 32 * 1024 * 1024;
-
-// Estimated upper bound on the compression ratio of compressed text files. Used to
-// estimate scanner thread memory usage.
-const int COMPRESSED_TEXT_COMPRESSION_RATIO = 11;
-
 // Amount of time to block waiting for GetNext() to release scanner threads between
 // checking if a scanner thread should yield itself back to the global thread pool.
 const int SCANNER_THREAD_WAIT_TIME_MS = 20;
@@ -63,11 +53,6 @@ const int SCANNER_THREAD_WAIT_TIME_MS = 20;
 HdfsScanNode::HdfsScanNode(ObjectPool* pool, const TPlanNode& tnode,
                            const DescriptorTbl& descs)
     : HdfsScanNodeBase(pool, tnode, descs),
-      ranges_issued_barrier_(1),
-      scanner_thread_bytes_required_(0),
-      done_(false),
-      all_ranges_started_(false),
-      thread_avail_cb_id_(-1),
       max_num_scanner_threads_(CpuInfo::num_cores()) {
 }
 
@@ -168,36 +153,6 @@ Status HdfsScanNode::Init(const TPlanNode& tnode, RuntimeState* state) {
 Status HdfsScanNode::Prepare(RuntimeState* state) {
   SCOPED_TIMER(runtime_profile_->total_time_counter());
   RETURN_IF_ERROR(HdfsScanNodeBase::Prepare(state));
-
-  // Compute the minimum bytes required to start a new thread. This is based on the
-  // file format.
-  // The higher the estimate, the less likely it is the query will fail but more likely
-  // the query will be throttled when it does not need to be.
-  // TODO: how many buffers should we estimate per range. The IoMgr will throttle down to
-  // one but if there are already buffers queued before memory pressure was hit, we can't
-  // reclaim that memory.
-  if (per_type_files_[THdfsFileFormat::PARQUET].size() > 0) {
-    // Parquet files require buffers per column
-    scanner_thread_bytes_required_ =
-        materialized_slots_.size() * 3 * runtime_state_->io_mgr()->max_buffer_size();
-  } else {
-    scanner_thread_bytes_required_ =
-        3 * runtime_state_->io_mgr()->max_buffer_size();
-  }
-  // scanner_thread_bytes_required_ now contains the IoBuffer requirement.
-  // Next we add in the other memory the scanner thread will use.
-  // e.g. decompression buffers, tuple buffers, etc.
-  // For compressed text, we estimate this based on the file size (since the whole file
-  // will need to be decompressed at once). For all other formats, we use a constant.
-  // TODO: can we do something better?
-  int64_t scanner_thread_mem_usage = SCANNER_THREAD_MEM_USAGE;
-  for (HdfsFileDesc* file: per_type_files_[THdfsFileFormat::TEXT]) {
-    if (file->file_compression != THdfsCompression::NONE) {
-      int64_t bytes_required = file->file_length * COMPRESSED_TEXT_COMPRESSION_RATIO;
-      scanner_thread_mem_usage = ::max(bytes_required, scanner_thread_mem_usage);
-    }
-  }
-  scanner_thread_bytes_required_ += scanner_thread_mem_usage;
   row_batches_get_timer_ = ADD_TIMER(runtime_profile(), "RowBatchQueueGetWaitTime");
   row_batches_put_timer_ = ADD_TIMER(runtime_profile(), "RowBatchQueuePutWaitTime");
   return Status::OK();
@@ -219,10 +174,9 @@ Status HdfsScanNode::Open(RuntimeState* state) {
     max_num_scanner_threads_ = runtime_state_->query_options().num_scanner_threads;
   }
   DCHECK_GT(max_num_scanner_threads_, 0);
-
+  spare_reservation_.Store(buffer_pool_client_.GetReservation());
   thread_avail_cb_id_ = runtime_state_->resource_pool()->AddThreadAvailableCb(
       bind<void>(mem_fn(&HdfsScanNode::ThreadTokenAvailableCb), this, _1));
-
   return Status::OK();
 }
 
@@ -263,37 +217,28 @@ Status HdfsScanNode::AddDiskIoRanges(const vector<ScanRange*>& ranges,
   return Status::OK();
 }
 
-// For controlling the amount of memory used for scanners, we approximate the
-// scanner mem usage based on scanner_thread_bytes_required_, rather than the
-// consumption in the scan node's mem tracker. The problem with the scan node
-// trackers is that it does not account for the memory the scanner will use.
-// For example, if there is 110 MB of memory left (based on the mem tracker)
-// and we estimate that a scanner will use 100MB, we want to make sure to only
-// start up one additional thread. However, after starting the first thread, the
-// mem tracker value will not change immediately (it takes some time before the
-// scanner is running and starts using memory). Therefore we just use the estimate
-// based on the number of running scanner threads.
-bool HdfsScanNode::EnoughMemoryForScannerThread(bool new_thread) {
-  int64_t committed_scanner_mem =
-      active_scanner_thread_counter_.value() * scanner_thread_bytes_required_;
-  int64_t tracker_consumption = mem_tracker()->consumption();
-  int64_t est_additional_scanner_mem = committed_scanner_mem - tracker_consumption;
-  if (est_additional_scanner_mem < 0) {
-    // This is the case where our estimate was too low. Expand the estimate based
-    // on the usage.
-    int64_t avg_consumption =
-        tracker_consumption / active_scanner_thread_counter_.value();
-    // Take the average and expand it by 50%. Some scanners will not have hit their
-    // steady state mem usage yet.
-    // TODO: how can we scale down if we've overestimated.
-    // TODO: better heuristic?
-    scanner_thread_bytes_required_ = static_cast<int64_t>(avg_consumption * 1.5);
-    est_additional_scanner_mem = 0;
-  }
+bool HdfsScanNode::EnoughReservationForExtraThread(const unique_lock<mutex>& lock) {
+  DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
+  if (spare_reservation_.Load() >= ideal_scan_range_reservation_) return true;
+  int64_t increase = ideal_scan_range_reservation_ - spare_reservation_.Load();
+  if (!buffer_pool_client_.IncreaseReservation(increase)) return false;
+  spare_reservation_.Add(increase);
+  return true;
+}
 
-  // If we are starting a new thread, take that into account now.
-  if (new_thread) est_additional_scanner_mem += scanner_thread_bytes_required_;
-  return est_additional_scanner_mem < mem_tracker()->SpareCapacity();
+int64_t HdfsScanNode::DeductReservationForScannerThread(const unique_lock<mutex>& lock,
+    bool first_thread) {
+  DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
+  int64_t amount;
+  if (first_thread) {
+    amount = spare_reservation_.Load() >= ideal_scan_range_reservation_ ?
+        ideal_scan_range_reservation_ : resource_profile_.min_reservation;
+  } else {
+    amount = ideal_scan_range_reservation_;
+  }
+  int64_t remainder = spare_reservation_.Add(-amount);
+  DCHECK_GE(remainder, 0);
+  return amount;
 }
 
 void HdfsScanNode::ThreadTokenAvailableCb(ThreadResourceMgr::ResourcePool* pool) {
@@ -323,36 +268,45 @@ void HdfsScanNode::ThreadTokenAvailableCb(ThreadResourceMgr::ResourcePool* pool)
     // TODO: This still leans heavily on starvation-free locks, come up with a more
     // correct way to communicate between this method and ScannerThreadHelper
     unique_lock<mutex> lock(lock_);
+
+    const int64_t num_active_scanner_threads = active_scanner_thread_counter_.value();
+    const bool first_thread = num_active_scanner_threads == 0;
     // Cases 1, 2, 3.
     if (done_ || all_ranges_started_ ||
-        active_scanner_thread_counter_.value() >= progress_.remaining()) {
+        num_active_scanner_threads >= progress_.remaining()) {
       break;
     }
 
     // Cases 5 and 6.
-    if (active_scanner_thread_counter_.value() > 0 &&
+    if (!first_thread &&
         (materialized_row_batches_->Size() >= max_materialized_row_batches_ ||
-         !EnoughMemoryForScannerThread(true))) {
+         !EnoughReservationForExtraThread(lock))) {
       break;
     }
 
     // Case 7 and 8.
-    bool is_reserved = false;
-    if (active_scanner_thread_counter_.value() >= max_num_scanner_threads_ ||
-        !pool->TryAcquireThreadToken(&is_reserved)) {
+    if (num_active_scanner_threads >= max_num_scanner_threads_ ||
+        !pool->TryAcquireThreadToken()) {
       break;
     }
 
+    // Deduct the reservation. We haven't dropped the lock since the
+    // first_thread/EnoughReservationForExtraThread() checks so spare reservation
+    // must be available.
+    int64_t scanner_thread_reservation =
+        DeductReservationForScannerThread(lock, first_thread);
     COUNTER_ADD(&active_scanner_thread_counter_, 1);
     string name = Substitute("scanner-thread (finst:$0, plan-node-id:$1, thread-idx:$2)",
         PrintId(runtime_state_->fragment_instance_id()), id(),
         num_scanner_threads_started_counter_->value());
-
-    auto fn = [this]() { this->ScannerThread(); };
+    auto fn = [this, scanner_thread_reservation]() {
+      this->ScannerThread(scanner_thread_reservation);
+    };
     std::unique_ptr<Thread> t;
     status =
       Thread::Create(FragmentInstanceState::FINST_THREAD_GROUP_NAME, name, fn, &t, true);
     if (!status.ok()) {
+      ReturnReservationFromScannerThread(scanner_thread_reservation);
       COUNTER_ADD(&active_scanner_thread_counter_, -1);
       // Release the token and skip running callbacks to find a replacement. Skipping
       // serves two purposes. First, it prevents a mutual recursion between this function
@@ -373,7 +327,7 @@ void HdfsScanNode::ThreadTokenAvailableCb(ThreadResourceMgr::ResourcePool* pool)
   }
 }
 
-void HdfsScanNode::ScannerThread() {
+void HdfsScanNode::ScannerThread(int64_t scanner_thread_reservation) {
   SCOPED_THREAD_COUNTER_MEASUREMENT(scanner_thread_counters());
   SCOPED_THREAD_COUNTER_MEASUREMENT(runtime_state_->total_thread_statistics());
   DiskIoMgr* io_mgr = runtime_state_->io_mgr();
@@ -402,8 +356,7 @@ void HdfsScanNode::ScannerThread() {
       // this thread.
       unique_lock<mutex> l(lock_);
       if (active_scanner_thread_counter_.value() > 1) {
-        if (runtime_state_->resource_pool()->optional_exceeded() ||
-            !EnoughMemoryForScannerThread(false)) {
+        if (runtime_state_->resource_pool()->optional_exceeded()) {
           // We can't break here. We need to update the counter with the lock held or else
           // all threads might see active_scanner_thread_counter_.value > 1
           COUNTER_ADD(&active_scanner_thread_counter_, -1);
@@ -438,12 +391,13 @@ void HdfsScanNode::ScannerThread() {
     if (status.ok() && scan_range != nullptr) {
       if (needs_buffers) {
         status = io_mgr->AllocateBuffersForRange(
-            reader_context_.get(), scan_range, 3 * io_mgr->max_buffer_size());
+            reader_context_.get(), &buffer_pool_client_, scan_range,
+            scanner_thread_reservation);
       }
       if (status.ok()) {
         // Got a scan range. Process the range end to end (in this thread).
         status = ProcessSplit(filter_status.ok() ? filter_ctxs : vector<FilterContext>(),
-            &expr_results_pool, scan_range);
+            &expr_results_pool, scan_range, scanner_thread_reservation);
       }
     }
 
@@ -483,6 +437,7 @@ void HdfsScanNode::ScannerThread() {
   COUNTER_ADD(&active_scanner_thread_counter_, -1);
 
 exit:
+  ReturnReservationFromScannerThread(scanner_thread_reservation);
   runtime_state_->resource_pool()->ReleaseThreadToken(false);
   for (auto& ctx: filter_ctxs) ctx.expr_eval->Close(runtime_state_);
   filter_mem_pool.FreeAll();
@@ -490,7 +445,8 @@ exit:
 }
 
 Status HdfsScanNode::ProcessSplit(const vector<FilterContext>& filter_ctxs,
-    MemPool* expr_results_pool, ScanRange* scan_range) {
+    MemPool* expr_results_pool, ScanRange* scan_range,
+    int64_t scanner_thread_reservation) {
   DCHECK(scan_range != NULL);
 
   ScanRangeMetadata* metadata = static_cast<ScanRangeMetadata*>(scan_range->meta_data());
@@ -515,8 +471,9 @@ Status HdfsScanNode::ProcessSplit(const vector<FilterContext>& filter_ctxs,
     return Status::OK();
   }
 
-  ScannerContext context(
-      runtime_state_, this, partition, scan_range, filter_ctxs, expr_results_pool);
+  ScannerContext context(runtime_state_, this, &buffer_pool_client_, partition,
+      filter_ctxs, expr_results_pool);
+  context.AddStream(scan_range, scanner_thread_reservation);
   scoped_ptr<HdfsScanner> scanner;
   Status status = CreateAndOpenScanner(partition, &context, &scanner);
   if (!status.ok()) {

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/hdfs-scan-node.h
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node.h b/be/src/exec/hdfs-scan-node.h
index a1c97cf..81e826e 100644
--- a/be/src/exec/hdfs-scan-node.h
+++ b/be/src/exec/hdfs-scan-node.h
@@ -27,6 +27,7 @@
 #include <boost/scoped_ptr.hpp>
 #include <boost/thread/mutex.hpp>
 
+#include "common/atomic.h"
 #include "exec/filter-context.h"
 #include "exec/hdfs-scan-node-base.h"
 #include "runtime/io/disk-io-mgr.h"
@@ -58,8 +59,14 @@ class TPlanNode;
 /// 5. The scanner finishes the scan range and informs the scan node so it can track
 ///    end of stream.
 ///
-/// TODO: This class allocates a bunch of small utility objects that should be
-/// recycled.
+/// Buffer management:
+/// ------------------
+/// The different scanner threads all allocate I/O buffers from the node's Buffer Pool
+/// client. The scan node ensures that enough reservation is available to start a
+/// scanner thread before launching each one with (see
+/// EnoughReservationForExtraThread()), after which the scanner thread is responsible
+/// for staying within the reservation handed off to it.
+///
 /// TODO: Remove this class once the fragment-based multi-threaded execution is
 /// fully functional.
 class HdfsScanNode : public HdfsScanNodeBase {
@@ -100,12 +107,7 @@ class HdfsScanNode : public HdfsScanNodeBase {
 
  private:
   /// Released when initial ranges are issued in the first call to GetNext().
-  CountingBarrier ranges_issued_barrier_;
-
-  /// The estimated memory required to start up a new scanner thread. If the memory
-  /// left (due to limits) is less than this value, we won't start up optional
-  /// scanner threads.
-  int64_t scanner_thread_bytes_required_;
+  CountingBarrier ranges_issued_barrier_{1};
 
   /// Thread group for all scanner worker threads
   ThreadGroup scanner_threads_;
@@ -130,16 +132,16 @@ class HdfsScanNode : public HdfsScanNodeBase {
   /// are finished, an error/cancellation occurred, or the limit was reached.
   /// Setting this to true triggers the scanner threads to clean up.
   /// This should not be explicitly set. Instead, call SetDone().
-  bool done_;
+  bool done_ = false;
 
   /// Set to true if all ranges have started. Some of the ranges may still be in flight
   /// being processed by scanner threads, but no new ScannerThreads should be started.
-  bool all_ranges_started_;
+  bool all_ranges_started_ = false;
 
   /// The id of the callback added to the thread resource manager when thread token
   /// is available. Used to remove the callback before this scan node is destroyed.
   /// -1 if no callback is registered.
-  int thread_avail_cb_id_;
+  int thread_avail_cb_id_ = -1;
 
   /// Maximum number of scanner threads. Set to 'NUM_SCANNER_THREADS' if that query
   /// option is set. Otherwise, it's set to the number of cpu cores. Scanner threads
@@ -147,6 +149,14 @@ class HdfsScanNode : public HdfsScanNodeBase {
   /// the number of cores.
   int max_num_scanner_threads_;
 
+  /// Amount of the 'buffer_pool_client_' reservation that is not allocated to scanner
+  /// threads. Doled out to scanner threads when they are started and returned when
+  /// those threads no longer need it. Can be atomically incremented without holding
+  /// 'lock_' but 'lock_' is held when decrementing to ensure that the check for
+  /// reservation and the actual deduction are atomic with respect to other threads
+  /// trying to claim reservation.
+  AtomicInt64 spare_reservation_{0};
+
   /// The wait time for fetching a row batch from the row batch queue.
   RuntimeProfile::Counter* row_batches_get_timer_;
 
@@ -160,21 +170,35 @@ class HdfsScanNode : public HdfsScanNodeBase {
   /// Main function for scanner thread. This thread pulls the next range to be
   /// processed from the IoMgr and then processes the entire range end to end.
   /// This thread terminates when all scan ranges are complete or an error occurred.
-  void ScannerThread();
+  /// The caller must have reserved 'scanner_thread_reservation' bytes of memory for
+  /// this thread with DeductReservationForScannerThread().
+  void ScannerThread(int64_t scanner_thread_reservation);
 
   /// Process the entire scan range with a new scanner object. Executed in scanner
   /// thread. 'filter_ctxs' is a clone of the class-wide filter_ctxs_, used to filter rows
   /// in this split.
   Status ProcessSplit(const std::vector<FilterContext>& filter_ctxs,
-      MemPool* expr_results_pool, io::ScanRange* scan_range) WARN_UNUSED_RESULT;
-
-  /// Returns true if there is enough memory (against the mem tracker limits) to
-  /// have a scanner thread.
-  /// If new_thread is true, the calculation is for starting a new scanner thread.
-  /// If false, it determines whether there's adequate memory for the existing
-  /// set of scanner threads.
-  /// lock_ must be taken before calling this.
-  bool EnoughMemoryForScannerThread(bool new_thread);
+      MemPool* expr_results_pool, io::ScanRange* scan_range,
+      int64_t scanner_thread_reservation) WARN_UNUSED_RESULT;
+
+  /// Return true if there is enough reservation to start an extra scanner thread.
+  /// Tries to increase reservation if enough is not already available in
+  /// 'spare_reservation_'. 'lock_' must be held via 'lock'
+  bool EnoughReservationForExtraThread(const boost::unique_lock<boost::mutex>& lock);
+
+  /// Deduct reservation to start a new scanner thread from 'spare_reservation_'. If
+  /// 'first_thread' is true, this is the first thread to be started and only the
+  /// minimum reservation is required to be available. Otherwise
+  /// EnoughReservationForExtra() thread must have returned true in the current
+  /// critical section so that 'ideal_scan_range_bytes_' is available for the extra
+  /// thread. Returns the amount deducted. 'lock_' must be held via 'lock'.
+  int64_t DeductReservationForScannerThread(const boost::unique_lock<boost::mutex>& lock,
+      bool first_thread);
+
+  /// Called by scanner thread to return or all of its reservation that is not needed.
+  void ReturnReservationFromScannerThread(int64_t bytes) {
+    spare_reservation_.Add(bytes);
+  }
 
   /// Checks for eos conditions and returns batches from materialized_row_batches_.
   Status GetNextInternal(RuntimeState* state, RowBatch* row_batch, bool* eos)

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/parquet-column-readers.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/parquet-column-readers.cc b/be/src/exec/parquet-column-readers.cc
index 099fdce..153b24a 100644
--- a/be/src/exec/parquet-column-readers.cc
+++ b/be/src/exec/parquet-column-readers.cc
@@ -30,6 +30,7 @@
 #include "exec/scanner-context.inline.h"
 #include "rpc/thrift-util.h"
 #include "runtime/collection-value-builder.h"
+#include "runtime/exec-env.h"
 #include "runtime/tuple-row.h"
 #include "runtime/tuple.h"
 #include "runtime/runtime-state.h"
@@ -47,9 +48,12 @@ DEFINE_bool(convert_legacy_hive_parquet_utc_timestamps, false,
     "When true, TIMESTAMPs read from files written by Parquet-MR (used by Hive) will "
     "be converted from UTC to local time. Writes are unaffected.");
 
-// Throttle deprecation warnings to - only print warning with this frequency.
 static const int BITPACKED_DEPRECATION_WARNING_FREQUENCY = 100;
 
+// Max dictionary page header size in bytes. This is an estimate and only needs to be an
+// upper bound.
+static const int MAX_DICT_HEADER_SIZE = 100;
+
 // Max data page header size in bytes. This is an estimate and only needs to be an upper
 // bound. It is theoretically possible to have a page header of any size due to string
 // value statistics, but in practice we'll have trouble reading string values this large.
@@ -69,6 +73,8 @@ static int debug_count = 0;
 #define SHOULD_TRIGGER_DEBUG_ACTION(x) (false)
 #endif
 
+using namespace impala::io;
+
 namespace impala {
 
 const string PARQUET_COL_MEM_LIMIT_EXCEEDED =
@@ -848,8 +854,99 @@ static bool RequiresSkippedDictionaryHeaderCheck(
   return v.VersionEq(1,1,0) || (v.VersionEq(1,2,0) && v.is_impala_internal);
 }
 
+Status BaseScalarColumnReader::Reset(const HdfsFileDesc& file_desc,
+    const parquet::ColumnChunk& col_chunk, int row_group_idx) {
+  num_buffered_values_ = 0;
+  data_ = nullptr;
+  data_end_ = nullptr;
+  stream_ = nullptr;
+  io_reservation_ = 0;
+  metadata_ = &col_chunk.meta_data;
+  num_values_read_ = 0;
+  def_level_ = -1;
+  // See ColumnReader constructor.
+  rep_level_ = max_rep_level() == 0 ? 0 : -1;
+  pos_current_value_ = -1;
+
+  if (metadata_->codec != parquet::CompressionCodec::UNCOMPRESSED) {
+    RETURN_IF_ERROR(Codec::CreateDecompressor(
+        nullptr, false, PARQUET_TO_IMPALA_CODEC[metadata_->codec], &decompressor_));
+  }
+  int64_t col_start = col_chunk.meta_data.data_page_offset;
+
+  RETURN_IF_ERROR(ParquetMetadataUtils::ValidateRowGroupColumn(parent_->file_metadata_,
+      parent_->filename(), row_group_idx, col_idx(), schema_element(),
+      parent_->state_));
+
+  if (col_chunk.meta_data.__isset.dictionary_page_offset) {
+    // Already validated in ValidateColumnOffsets()
+    DCHECK_LT(col_chunk.meta_data.dictionary_page_offset, col_start);
+    col_start = col_chunk.meta_data.dictionary_page_offset;
+  }
+  int64_t col_len = col_chunk.meta_data.total_compressed_size;
+  if (col_len <= 0) {
+    return Status(Substitute("File '$0' contains invalid column chunk size: $1",
+        filename(), col_len));
+  }
+  int64_t col_end = col_start + col_len;
+
+  // Already validated in ValidateColumnOffsets()
+  DCHECK_GT(col_end, 0);
+  DCHECK_LT(col_end, file_desc.file_length);
+  const ParquetFileVersion& file_version = parent_->file_version_;
+  if (file_version.application == "parquet-mr" && file_version.VersionLt(1, 2, 9)) {
+    // The Parquet MR writer had a bug in 1.2.8 and below where it didn't include the
+    // dictionary page header size in total_compressed_size and total_uncompressed_size
+    // (see IMPALA-694). We pad col_len to compensate.
+    int64_t bytes_remaining = file_desc.file_length - col_end;
+    int64_t pad = min<int64_t>(MAX_DICT_HEADER_SIZE, bytes_remaining);
+    col_len += pad;
+  }
+
+  // TODO: this will need to change when we have co-located files and the columns
+  // are different files.
+  if (!col_chunk.file_path.empty() && col_chunk.file_path != filename()) {
+    return Status(Substitute("Expected parquet column file path '$0' to match "
+        "filename '$1'", col_chunk.file_path, filename()));
+  }
+
+  const ScanRange* metadata_range = parent_->metadata_range_;
+  int64_t partition_id = parent_->context_->partition_descriptor()->id();
+  const ScanRange* split_range =
+      static_cast<ScanRangeMetadata*>(metadata_range->meta_data())->original_split;
+  // Determine if the column is completely contained within a local split.
+  bool col_range_local = split_range->expected_local()
+      && col_start >= split_range->offset()
+      && col_end <= split_range->offset() + split_range->len();
+  scan_range_ = parent_->scan_node_->AllocateScanRange(metadata_range->fs(),
+      filename(), col_len, col_start, partition_id, split_range->disk_id(),
+      col_range_local,
+      BufferOpts(split_range->try_cache(), file_desc.mtime));
+  ClearDictionaryDecoder();
+  return Status::OK();
+}
+
+Status BaseScalarColumnReader::StartScan() {
+  DCHECK(scan_range_ != nullptr) << "Must Reset() before starting scan.";
+  DiskIoMgr* io_mgr = ExecEnv::GetInstance()->disk_io_mgr();
+  ScannerContext* context = parent_->context_;
+  DCHECK_GT(io_reservation_, 0);
+  bool needs_buffers;
+  RETURN_IF_ERROR(io_mgr->StartScanRange(
+      parent_->scan_node_->reader_context(), scan_range_, &needs_buffers));
+  if (needs_buffers) {
+    RETURN_IF_ERROR(io_mgr->AllocateBuffersForRange(
+        parent_->scan_node_->reader_context(), context->bp_client(),
+        scan_range_, io_reservation_));
+  }
+  stream_ = parent_->context_->AddStream(scan_range_, io_reservation_);
+  DCHECK(stream_ != nullptr);
+  return Status::OK();
+}
+
 Status BaseScalarColumnReader::ReadPageHeader(bool peek,
     parquet::PageHeader* next_page_header, uint32_t* next_header_size, bool* eos) {
+  DCHECK(stream_ != nullptr);
   *eos = false;
 
   uint8_t* buffer;
@@ -933,7 +1030,7 @@ Status BaseScalarColumnReader::InitDictionary() {
   bool eos;
   parquet::PageHeader next_page_header;
   uint32_t next_header_size;
-
+  DCHECK(stream_ != nullptr);
   DCHECK(!HasDictionaryDecoder());
 
   RETURN_IF_ERROR(ReadPageHeader(true /* peek */, &next_page_header,
@@ -1045,6 +1142,14 @@ Status BaseScalarColumnReader::InitDictionary() {
   return Status::OK();
 }
 
+Status BaseScalarColumnReader::InitDictionaries(
+    const vector<BaseScalarColumnReader*> readers) {
+  for (BaseScalarColumnReader* reader : readers) {
+    RETURN_IF_ERROR(reader->InitDictionary());
+  }
+  return Status::OK();
+}
+
 Status BaseScalarColumnReader::ReadDataPage() {
   // We're about to move to the next data page.  The previous data page is
   // now complete, free up any memory allocated for it. If the data page contained

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/parquet-column-readers.h
----------------------------------------------------------------------
diff --git a/be/src/exec/parquet-column-readers.h b/be/src/exec/parquet-column-readers.h
index 3a8ad70..bfb20ec 100644
--- a/be/src/exec/parquet-column-readers.h
+++ b/be/src/exec/parquet-column-readers.h
@@ -326,42 +326,29 @@ class BaseScalarColumnReader : public ParquetColumnReader {
   BaseScalarColumnReader(HdfsParquetScanner* parent, const SchemaNode& node,
       const SlotDescriptor* slot_desc)
     : ParquetColumnReader(parent, node, slot_desc),
-      data_(NULL),
-      data_end_(NULL),
-      def_levels_(true),
-      rep_levels_(false),
-      page_encoding_(parquet::Encoding::PLAIN_DICTIONARY),
-      num_buffered_values_(0),
-      num_values_read_(0),
-      metadata_(NULL),
-      stream_(NULL),
       data_page_pool_(new MemPool(parent->scan_node_->mem_tracker())) {
     DCHECK_GE(node_.col_idx, 0) << node_.DebugString();
   }
 
   virtual ~BaseScalarColumnReader() { }
 
-  /// This is called once for each row group in the file.
-  Status Reset(const parquet::ColumnMetaData* metadata, ScannerContext::Stream* stream) {
-    DCHECK(stream != NULL);
-    DCHECK(metadata != NULL);
-
-    num_buffered_values_ = 0;
-    data_ = NULL;
-    data_end_ = NULL;
-    stream_ = stream;
-    metadata_ = metadata;
-    num_values_read_ = 0;
-    def_level_ = -1;
-    // See ColumnReader constructor.
-    rep_level_ = max_rep_level() == 0 ? 0 : -1;
-    pos_current_value_ = -1;
-
-    if (metadata_->codec != parquet::CompressionCodec::UNCOMPRESSED) {
-      RETURN_IF_ERROR(Codec::CreateDecompressor(
-          NULL, false, PARQUET_TO_IMPALA_CODEC[metadata_->codec], &decompressor_));
-    }
-    ClearDictionaryDecoder();
+  /// Resets the reader for each row group in the file and creates the scan
+  /// range for the column, but does not start it. To start scanning,
+  /// set_io_reservation() must be called to assign reservation to this
+  /// column, followed by StartScan().
+  Status Reset(const HdfsFileDesc& file_desc, const parquet::ColumnChunk& col_chunk,
+    int row_group_idx);
+
+  /// Starts the column scan range. The reader must be Reset() and have a
+  /// reservation assigned via set_io_reservation(). This must be called
+  /// before any of the column data can be read (including dictionary and
+  /// data pages). Returns an error status if there was an error starting the
+  /// scan or allocating buffers for it.
+  Status StartScan();
+
+  /// Helper to start scans for multiple columns at once.
+  static Status StartScans(const std::vector<BaseScalarColumnReader*> readers) {
+    for (BaseScalarColumnReader* reader : readers) RETURN_IF_ERROR(reader->StartScan());
     return Status::OK();
   }
 
@@ -376,22 +363,27 @@ class BaseScalarColumnReader : public ParquetColumnReader {
     if (dict_decoder != nullptr) dict_decoder->Close();
   }
 
+  io::ScanRange* scan_range() const { return scan_range_; }
   int64_t total_len() const { return metadata_->total_compressed_size; }
   int col_idx() const { return node_.col_idx; }
   THdfsCompression::type codec() const {
     if (metadata_ == NULL) return THdfsCompression::NONE;
     return PARQUET_TO_IMPALA_CODEC[metadata_->codec];
   }
+  void set_io_reservation(int bytes) { io_reservation_ = bytes; }
 
   /// Reads the next definition and repetition levels for this column. Initializes the
   /// next data page if necessary.
   virtual bool NextLevels() { return NextLevels<true>(); }
 
-  // Check the data stream to see if there is a dictionary page. If there is,
-  // use that page to initialize dict_decoder_ and advance the data stream
-  // past the dictionary page.
+  /// Check the data stream to see if there is a dictionary page. If there is,
+  /// use that page to initialize dict_decoder_ and advance the data stream
+  /// past the dictionary page.
   Status InitDictionary();
 
+  /// Convenience function to initialize multiple dictionaries.
+  static Status InitDictionaries(const std::vector<BaseScalarColumnReader*> readers);
+
   // Returns the dictionary or NULL if the dictionary doesn't exist
   virtual DictDecoderBase* GetDictionaryDecoder() { return nullptr; }
 
@@ -417,33 +409,45 @@ class BaseScalarColumnReader : public ParquetColumnReader {
   // fit in as few cache lines as possible.
 
   /// Pointer to start of next value in data page
-  uint8_t* data_;
+  uint8_t* data_ = nullptr;
 
   /// End of the data page.
-  const uint8_t* data_end_;
+  const uint8_t* data_end_ = nullptr;
 
   /// Decoder for definition levels.
-  ParquetLevelDecoder def_levels_;
+  ParquetLevelDecoder def_levels_{true};
 
   /// Decoder for repetition levels.
-  ParquetLevelDecoder rep_levels_;
+  ParquetLevelDecoder rep_levels_{false};
 
   /// Page encoding for values of the current data page. Cached here for perf. Set in
   /// InitDataPage().
-  parquet::Encoding::type page_encoding_;
+  parquet::Encoding::type page_encoding_ = parquet::Encoding::PLAIN_DICTIONARY;
 
   /// Num values remaining in the current data page
-  int num_buffered_values_;
+  int num_buffered_values_ = 0;
 
   // Less frequently used members that are not accessed in inner loop should go below
   // here so they do not occupy precious cache line space.
 
   /// The number of values seen so far. Updated per data page.
-  int64_t num_values_read_;
+  int64_t num_values_read_ = 0;
+
+  /// Metadata for the column for the current row group.
+  const parquet::ColumnMetaData* metadata_ = nullptr;
 
-  const parquet::ColumnMetaData* metadata_;
   boost::scoped_ptr<Codec> decompressor_;
-  ScannerContext::Stream* stream_;
+
+  /// The scan range for the column's data. Initialized for each row group by Reset().
+  io::ScanRange* scan_range_ = nullptr;
+
+  // Stream used to read data from 'scan_range_'. Initialized by StartScan().
+  ScannerContext::Stream* stream_ = nullptr;
+
+  /// Reservation in bytes to use for I/O buffers in 'scan_range_'/'stream_'. Must be set
+  /// with set_io_reservation() before 'stream_' is initialized. Reset for each row group
+  /// by Reset().
+  int64_t io_reservation_ = 0;
 
   /// Pool to allocate storage for data pages from - either decompression buffers for
   /// compressed data pages or copies of the data page with var-len data to attach to

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/scanner-context.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/scanner-context.cc b/be/src/exec/scanner-context.cc
index 0abf82f..c669e65 100644
--- a/be/src/exec/scanner-context.cc
+++ b/be/src/exec/scanner-context.cc
@@ -41,14 +41,15 @@ static const int64_t INIT_READ_PAST_SIZE_BYTES = 64 * 1024;
 const int64_t ScannerContext::Stream::OUTPUT_BUFFER_BYTES_LEFT_INIT;
 
 ScannerContext::ScannerContext(RuntimeState* state, HdfsScanNodeBase* scan_node,
-    HdfsPartitionDescriptor* partition_desc, ScanRange* scan_range,
-    const vector<FilterContext>& filter_ctxs, MemPool* expr_results_pool)
+    BufferPool::ClientHandle* bp_client, HdfsPartitionDescriptor* partition_desc,
+    const vector<FilterContext>& filter_ctxs,
+    MemPool* expr_results_pool)
   : state_(state),
     scan_node_(scan_node),
+    bp_client_(bp_client),
     partition_desc_(partition_desc),
     filter_ctxs_(filter_ctxs),
     expr_results_pool_(expr_results_pool) {
-  AddStream(scan_range);
 }
 
 ScannerContext::~ScannerContext() {
@@ -66,19 +67,20 @@ void ScannerContext::ClearStreams() {
 }
 
 ScannerContext::Stream::Stream(ScannerContext* parent, ScanRange* scan_range,
-    const HdfsFileDesc* file_desc)
+    int64_t reservation, const HdfsFileDesc* file_desc)
   : parent_(parent),
     scan_range_(scan_range),
     file_desc_(file_desc),
+    reservation_(reservation),
     file_len_(file_desc->file_length),
     next_read_past_size_bytes_(INIT_READ_PAST_SIZE_BYTES),
     boundary_pool_(new MemPool(parent->scan_node_->mem_tracker())),
     boundary_buffer_(new StringBuffer(boundary_pool_.get())) {
 }
 
-ScannerContext::Stream* ScannerContext::AddStream(ScanRange* range) {
-  streams_.emplace_back(new Stream(
-      this, range, scan_node_->GetFileDesc(partition_desc_->id(), range->file())));
+ScannerContext::Stream* ScannerContext::AddStream(ScanRange* range, int64_t reservation) {
+  streams_.emplace_back(new Stream(this, range, reservation,
+      scan_node_->GetFileDesc(partition_desc_->id(), range->file())));
   return streams_.back().get();
 }
 
@@ -101,7 +103,7 @@ void ScannerContext::Stream::ReleaseCompletedResources(bool done) {
 
 Status ScannerContext::Stream::GetNextBuffer(int64_t read_past_size) {
   DCHECK_EQ(0, io_buffer_bytes_left_);
-  DiskIoMgr* io_mgr = parent_->state_->io_mgr();
+  DiskIoMgr* io_mgr = ExecEnv::GetInstance()->disk_io_mgr();
   if (UNLIKELY(parent_->cancelled())) return Status::CANCELLED;
   if (io_buffer_ != nullptr) ReturnIoBuffer();
 
@@ -134,6 +136,7 @@ Status ScannerContext::Stream::GetNextBuffer(int64_t read_past_size) {
     read_past_buffer_size = ::max(read_past_buffer_size, read_past_size);
     read_past_buffer_size = ::min(read_past_buffer_size, file_bytes_remaining);
     read_past_buffer_size = ::min(read_past_buffer_size, max_buffer_size);
+    read_past_buffer_size = ::min(read_past_buffer_size, reservation_);
     // We're reading past the scan range. Be careful not to read past the end of file.
     DCHECK_GE(read_past_buffer_size, 0);
     if (read_past_buffer_size == 0) {
@@ -150,9 +153,14 @@ Status ScannerContext::Stream::GetNextBuffer(int64_t read_past_size) {
     if (needs_buffers) {
       // Allocate fresh buffers. The buffers for 'scan_range_' should be released now
       // since we hit EOS.
+      if (reservation_ < io_mgr->min_buffer_size()) {
+        return Status(Substitute("Could not read past end of scan range in file '$0'. "
+            "Reservation provided $1 was < the minimum I/O buffer size",
+            reservation_, io_mgr->min_buffer_size()));
+      }
       RETURN_IF_ERROR(io_mgr->AllocateBuffersForRange(
-          parent_->scan_node_->reader_context(), range,
-          3 * io_mgr->max_buffer_size()));
+          parent_->scan_node_->reader_context(), parent_->bp_client_, range,
+          reservation_));
     }
     RETURN_IF_ERROR(range->GetNext(&io_buffer_));
     DCHECK(io_buffer_->eosr());

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/exec/scanner-context.h
----------------------------------------------------------------------
diff --git a/be/src/exec/scanner-context.h b/be/src/exec/scanner-context.h
index a131d3f..6292486 100644
--- a/be/src/exec/scanner-context.h
+++ b/be/src/exec/scanner-context.h
@@ -27,6 +27,7 @@
 #include "common/compiler-util.h"
 #include "common/status.h"
 #include "exec/filter-context.h"
+#include "runtime/bufferpool/buffer-pool.h"
 #include "runtime/io/request-ranges.h"
 
 namespace impala {
@@ -84,10 +85,12 @@ class TupleRow;
 class ScannerContext {
  public:
   /// Create a scanner context with the parent scan_node (where materialized row batches
-  /// get pushed to) and the scan range to process.
-  /// This context starts with 1 stream.
-  ScannerContext(RuntimeState*, HdfsScanNodeBase*, HdfsPartitionDescriptor*,
-      io::ScanRange* scan_range, const std::vector<FilterContext>& filter_ctxs,
+  /// get pushed to) and the scan range to process. Buffers are allocated using
+  /// 'bp_client'.
+  ScannerContext(RuntimeState* state, HdfsScanNodeBase* scan_node,
+      BufferPool::ClientHandle* bp_client,
+      HdfsPartitionDescriptor* partition_desc,
+      const std::vector<FilterContext>& filter_ctxs,
       MemPool* expr_results_pool);
   /// Destructor verifies that all stream objects have been released.
   ~ScannerContext();
@@ -150,6 +153,7 @@ class ScannerContext {
     const char* filename() { return scan_range_->file(); }
     const io::ScanRange* scan_range() { return scan_range_; }
     const HdfsFileDesc* file_desc() { return file_desc_; }
+    int64_t reservation() const { return reservation_; }
 
     /// Returns the buffer's current offset in the file.
     int64_t file_offset() const { return scan_range_->offset() + total_bytes_returned_; }
@@ -211,9 +215,15 @@ class ScannerContext {
 
    private:
     friend class ScannerContext;
-    ScannerContext* parent_;
-    io::ScanRange* scan_range_;
-    const HdfsFileDesc* file_desc_;
+    ScannerContext* const parent_;
+    io::ScanRange* const scan_range_;
+    const HdfsFileDesc* const file_desc_;
+
+    /// Reservation given to this stream for allocating I/O buffers. The reservation is
+    /// shared with 'scan_range_', so the context must be careful not to use this until
+    /// all of 'scan_ranges_'s buffers have been freed. Must be >= the minimum IoMgr
+    /// buffer size to allow reading past the end of 'scan_range_'.
+    const int64_t reservation_;
 
     /// Total number of bytes returned from GetBytes()
     int64_t total_bytes_returned_ = 0;
@@ -272,7 +282,8 @@ class ScannerContext {
     /// output_buffer_bytes_left_ will be set to something else.
     static const int64_t OUTPUT_BUFFER_BYTES_LEFT_INIT = 0;
 
-    Stream(ScannerContext* parent, io::ScanRange* scan_range,
+    /// Private constructor. See AddStream() for public API.
+    Stream(ScannerContext* parent, io::ScanRange* scan_range, int64_t reservation,
         const HdfsFileDesc* file_desc);
 
     /// GetBytes helper to handle the slow path.
@@ -355,24 +366,37 @@ class ScannerContext {
   /// size to 0.
   void ClearStreams();
 
-  /// Add a stream to this ScannerContext for 'range'. The stream is owned by this
-  /// context.
-  Stream* AddStream(io::ScanRange* range);
+  /// Add a stream to this ScannerContext for 'range'. 'range' must already have any
+  /// buffers that it needs allocated. 'reservation' is the amount of reservation that
+  /// is given to this stream for allocating I/O buffers. The reservation is shared with
+  /// 'range', so the context must be careful not to use this until all of 'range's
+  /// buffers have been freed. Must be >= the minimum IoMgr buffer size o allow reading
+  /// past the end of 'range'.
+  ///
+  /// Returns the added stream. The returned stream is owned by this context.
+  Stream* AddStream(io::ScanRange* range, int64_t reservation);
 
   /// Returns true if RuntimeState::is_cancelled() is true, or if scan node is not
   /// multi-threaded and is done (finished, cancelled or reached it's limit).
   /// In all other cases returns false.
   bool cancelled() const;
 
-  HdfsPartitionDescriptor* partition_descriptor() { return partition_desc_; }
+  BufferPool::ClientHandle* bp_client() const { return bp_client_; }
+  HdfsPartitionDescriptor* partition_descriptor() const { return partition_desc_; }
   const std::vector<FilterContext>& filter_ctxs() const { return filter_ctxs_; }
   MemPool* expr_results_pool() const { return expr_results_pool_; }
  private:
   friend class Stream;
 
-  RuntimeState* state_;
-  HdfsScanNodeBase* scan_node_;
-  HdfsPartitionDescriptor* partition_desc_;
+  RuntimeState* const state_;
+  HdfsScanNodeBase* const scan_node_;
+
+  /// Buffer pool client used to allocate I/O buffers. This is accessed by multiple
+  /// threads in the multi-threaded scan node, so those threads must take care to only
+  /// call thread-safe BufferPool methods with this client.
+  BufferPool::ClientHandle* const bp_client_;
+
+  HdfsPartitionDescriptor* const partition_desc_;
 
   /// Vector of streams. Non-columnar formats will always have one stream per context.
   std::vector<std::unique_ptr<Stream>> streams_;

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/bufferpool/reservation-tracker-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/reservation-tracker-test.cc b/be/src/runtime/bufferpool/reservation-tracker-test.cc
index c46c5ea..e441402 100644
--- a/be/src/runtime/bufferpool/reservation-tracker-test.cc
+++ b/be/src/runtime/bufferpool/reservation-tracker-test.cc
@@ -522,15 +522,15 @@ TEST_F(ReservationTrackerTest, TransferReservation) {
 TEST_F(ReservationTrackerTest, ReservationUtil) {
   const int64_t MEG = 1024 * 1024;
   const int64_t GIG = 1024 * 1024 * 1024;
-  EXPECT_EQ(75 * MEG, ReservationUtil::RESERVATION_MEM_MIN_REMAINING);
+  EXPECT_EQ(32 * MEG, ReservationUtil::RESERVATION_MEM_MIN_REMAINING);
 
   EXPECT_EQ(0, ReservationUtil::GetReservationLimitFromMemLimit(0));
   EXPECT_EQ(0, ReservationUtil::GetReservationLimitFromMemLimit(-1));
-  EXPECT_EQ(0, ReservationUtil::GetReservationLimitFromMemLimit(75 * MEG));
+  EXPECT_EQ(0, ReservationUtil::GetReservationLimitFromMemLimit(32 * MEG));
   EXPECT_EQ(8 * GIG, ReservationUtil::GetReservationLimitFromMemLimit(10 * GIG));
 
-  EXPECT_EQ(75 * MEG, ReservationUtil::GetMinMemLimitFromReservation(0));
-  EXPECT_EQ(75 * MEG, ReservationUtil::GetMinMemLimitFromReservation(-1));
+  EXPECT_EQ(32 * MEG, ReservationUtil::GetMinMemLimitFromReservation(0));
+  EXPECT_EQ(32 * MEG, ReservationUtil::GetMinMemLimitFromReservation(-1));
   EXPECT_EQ(500 * MEG, ReservationUtil::GetMinMemLimitFromReservation(400 * MEG));
   EXPECT_EQ(5 * GIG, ReservationUtil::GetMinMemLimitFromReservation(4 * GIG));
 

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/bufferpool/reservation-util.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/bufferpool/reservation-util.cc b/be/src/runtime/bufferpool/reservation-util.cc
index 85718ab..a27ab9d 100644
--- a/be/src/runtime/bufferpool/reservation-util.cc
+++ b/be/src/runtime/bufferpool/reservation-util.cc
@@ -24,7 +24,7 @@ namespace impala {
 // Most operators that accumulate memory use reservations, so the majority of memory
 // should be allocated to buffer reservations, as a heuristic.
 const double ReservationUtil::RESERVATION_MEM_FRACTION = 0.8;
-const int64_t ReservationUtil::RESERVATION_MEM_MIN_REMAINING = 75 * 1024 * 1024;
+const int64_t ReservationUtil::RESERVATION_MEM_MIN_REMAINING = 32 * 1024 * 1024;
 
 int64_t ReservationUtil::GetReservationLimitFromMemLimit(int64_t mem_limit) {
   int64_t max_reservation = std::min<int64_t>(

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/disk-io-mgr-stress-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-stress-test.cc b/be/src/runtime/io/disk-io-mgr-stress-test.cc
index 0e41a6f..2ec1d09 100644
--- a/be/src/runtime/io/disk-io-mgr-stress-test.cc
+++ b/be/src/runtime/io/disk-io-mgr-stress-test.cc
@@ -15,9 +15,11 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "runtime/io/disk-io-mgr-stress.h"
+#include <gflags/gflags.h>
 
 #include "common/init.h"
+#include "runtime/io/disk-io-mgr-stress.h"
+#include "common/init.h"
 #include "runtime/test-env.h"
 #include "service/fe-support.h"
 #include "util/string-parser.h"
@@ -31,34 +33,32 @@ using namespace impala::io;
 // can be passed to control how long to run this test (0 for forever).
 
 // TODO: make these configurable once we decide how to run BE tests with args
-const int DEFAULT_DURATION_SEC = 1;
+constexpr int DEFAULT_DURATION_SEC = 1;
 const int NUM_DISKS = 5;
 const int NUM_THREADS_PER_DISK = 5;
 const int NUM_CLIENTS = 10;
 const bool TEST_CANCELLATION = true;
+const int64_t BUFFER_POOL_CAPACITY = 1024L * 1024L * 1024L * 4L;
+
+DEFINE_int64(duration_sec, DEFAULT_DURATION_SEC,
+    "Disk I/O Manager stress test duration in seconds. 0 means run indefinitely.");
 
 int main(int argc, char** argv) {
-  InitCommonRuntime(argc, argv, true, TestInfo::BE_TEST);
-  InitFeSupport();
-  TestEnv test_env;
-  ABORT_IF_ERROR(test_env.Init());
-  int duration_sec = DEFAULT_DURATION_SEC;
+  impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
+  impala::InitFeSupport();
 
-  if (argc == 2) {
-    StringParser::ParseResult status;
-    duration_sec = StringParser::StringToInt<int>(argv[1], strlen(argv[1]), &status);
-    if (status != StringParser::PARSE_SUCCESS) {
-      printf("Invalid arg: %s\n", argv[1]);
-      return 1;
-    }
-  }
-  if (duration_sec != 0) {
-    printf("Running stress test for %d seconds.\n", duration_sec);
+  if (FLAGS_duration_sec != 0) {
+    printf("Running stress test for %ld seconds.\n", FLAGS_duration_sec);
   } else {
     printf("Running stress test indefinitely.\n");
   }
-  DiskIoMgrStress test(NUM_DISKS, NUM_THREADS_PER_DISK, NUM_CLIENTS, TEST_CANCELLATION);
-  test.Run(duration_sec);
 
+  TestEnv test_env;
+  // Tests try to allocate arbitrarily small buffers. Ensure Buffer Pool allows it.
+  test_env.SetBufferPoolArgs(DiskIoMgrStress::MIN_READ_BUFFER_SIZE, BUFFER_POOL_CAPACITY);
+  Status status = test_env.Init();
+  CHECK(status.ok()) << status.GetDetail();
+  DiskIoMgrStress test(NUM_DISKS, NUM_THREADS_PER_DISK, NUM_CLIENTS, TEST_CANCELLATION);
+  test.Run(FLAGS_duration_sec);
   return 0;
 }


[13/15] impala git commit: IMPALA-4835: Part 2: Allocate scan range buffers upfront

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/request-context.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-context.cc b/be/src/runtime/io/request-context.cc
index 031b976..b124702 100644
--- a/be/src/runtime/io/request-context.cc
+++ b/be/src/runtime/io/request-context.cc
@@ -36,30 +36,6 @@ BufferDescriptor::BufferDescriptor(DiskIoMgr* io_mgr,
   DCHECK_GE(buffer_len, 0);
 }
 
-Status RequestContext::AllocBuffer(ScanRange* range, int64_t buffer_size,
-    unique_ptr<BufferDescriptor>* buffer_desc) {
-  DCHECK(range->external_buffer_tag_ == ScanRange::ExternalBufferTag::NO_BUFFER)
-      << static_cast<int>(range->external_buffer_tag_);
-  DCHECK_LE(buffer_size, parent_->max_buffer_size_);
-  DCHECK_GT(buffer_size, 0);
-  buffer_size = BitUtil::RoundUpToPowerOfTwo(
-      max(parent_->min_buffer_size_, min(parent_->max_buffer_size_, buffer_size)));
-
-  DCHECK(mem_tracker_ != nullptr);
-  if (!mem_tracker_->TryConsume(buffer_size)) {
-    return mem_tracker_->MemLimitExceeded(nullptr, "disk I/O buffer", buffer_size);
-  }
-
-  uint8_t* buffer = reinterpret_cast<uint8_t*>(malloc(buffer_size));
-  if (buffer == nullptr) {
-    mem_tracker_->Release(buffer_size);
-    return Status(TErrorCode::INTERNAL_ERROR,
-        Substitute("Could not malloc buffer of $0 bytes"));
-  }
-  buffer_desc->reset(new BufferDescriptor(parent_, this, range, buffer, buffer_size));
-  return Status::OK();
-}
-
 void RequestContext::FreeBuffer(BufferDescriptor* buffer) {
   DCHECK(buffer->buffer_ != nullptr);
   if (!buffer->is_cached() && !buffer->is_client_buffer()) {
@@ -108,40 +84,26 @@ void RequestContext::Cancel() {
     // The reader will be put into a cancelled state until call cleanup is complete.
     state_ = RequestContext::Cancelled;
 
-    // Cancel all scan ranges for this reader. Each range could be one one of
-    // four queues.
-    for (int i = 0; i < disk_states_.size(); ++i) {
-      PerDiskState& state = disk_states_[i];
-      RequestRange* range = nullptr;
-      while ((range = state.in_flight_ranges()->Dequeue()) != nullptr) {
-        if (range->request_type() == RequestType::READ) {
-          static_cast<ScanRange*>(range)->Cancel(Status::CANCELLED);
-        } else {
-          DCHECK(range->request_type() == RequestType::WRITE);
+    // Clear out all request ranges from queues for this reader. Cancel the scan
+    // ranges and invoke the write range callbacks to propagate the cancellation.
+    for (ScanRange* range : active_scan_ranges_) range->CancelInternal(Status::CANCELLED);
+    active_scan_ranges_.clear();
+    for (PerDiskState& disk_state : disk_states_) {
+      RequestRange* range;
+      while ((range = disk_state.in_flight_ranges()->Dequeue()) != nullptr) {
+        if (range->request_type() == RequestType::WRITE) {
           write_callbacks.push_back(static_cast<WriteRange*>(range)->callback_);
         }
       }
-
-      ScanRange* scan_range;
-      while ((scan_range = state.unstarted_scan_ranges()->Dequeue()) != nullptr) {
-        scan_range->Cancel(Status::CANCELLED);
-      }
+      while (disk_state.unstarted_scan_ranges()->Dequeue() != nullptr);
       WriteRange* write_range;
-      while ((write_range = state.unstarted_write_ranges()->Dequeue()) != nullptr) {
+      while ((write_range = disk_state.unstarted_write_ranges()->Dequeue()) != nullptr) {
         write_callbacks.push_back(write_range->callback_);
       }
     }
-
-    ScanRange* range = nullptr;
-    while ((range = ready_to_start_ranges_.Dequeue()) != nullptr) {
-      range->Cancel(Status::CANCELLED);
-    }
-    while ((range = blocked_ranges_.Dequeue()) != nullptr) {
-      range->Cancel(Status::CANCELLED);
-    }
-    while ((range = cached_ranges_.Dequeue()) != nullptr) {
-      range->Cancel(Status::CANCELLED);
-    }
+    // Clear out the lists of scan ranges.
+    while (ready_to_start_ranges_.Dequeue() != nullptr);
+    while (cached_ranges_.Dequeue() != nullptr);
 
     // Ensure that the reader is scheduled on all disks (it may already be scheduled on
     // some). The disk threads will notice that the context is cancelled and do any
@@ -170,9 +132,8 @@ void RequestContext::CancelAndMarkInactive() {
   // Wait until the ranges finish up.
   while (num_disks_with_ranges_ > 0) disks_complete_cond_var_.Wait(l);
 
-  // Validate that no buffers were leaked from this context.
-  DCHECK_EQ(num_buffers_in_reader_.Load(), 0) << endl << DebugString();
-  DCHECK_EQ(num_used_buffers_.Load(), 0) << endl << DebugString();
+  // Validate that no ranges are active.
+  DCHECK_EQ(0, active_scan_ranges_.size()) << endl << DebugString();
 
   // Validate that no threads are active and the context is not queued.
   for (const PerDiskState& disk_state : disk_states_) {
@@ -185,42 +146,58 @@ void RequestContext::CancelAndMarkInactive() {
   state_ = Inactive;
 }
 
-void RequestContext::AddRequestRange(const unique_lock<mutex>& lock,
-    RequestRange* range, bool schedule_immediately) {
+void RequestContext::AddRangeToDisk(const unique_lock<mutex>& lock,
+    RequestRange* range, ScheduleMode schedule_mode) {
   DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
-  PerDiskState& state = disk_states_[range->disk_id()];
-  if (state.done()) {
-    DCHECK_EQ(state.num_remaining_ranges(), 0);
-    state.set_done(false);
+  DCHECK_EQ(state_, Active) << DebugString();
+  PerDiskState* disk_state = &disk_states_[range->disk_id()];
+  if (disk_state->done()) {
+    DCHECK_EQ(disk_state->num_remaining_ranges(), 0);
+    disk_state->set_done(false);
     ++num_disks_with_ranges_;
   }
-
-  bool schedule_context;
   if (range->request_type() == RequestType::READ) {
     ScanRange* scan_range = static_cast<ScanRange*>(range);
-    if (schedule_immediately) {
+    if (schedule_mode == ScheduleMode::IMMEDIATELY) {
       ScheduleScanRange(lock, scan_range);
-    } else {
-      state.unstarted_scan_ranges()->Enqueue(scan_range);
+    } else if (schedule_mode == ScheduleMode::UPON_GETNEXT) {
+      disk_state->unstarted_scan_ranges()->Enqueue(scan_range);
       num_unstarted_scan_ranges_.Add(1);
+      // If there's no 'next_scan_range_to_start', schedule this RequestContext so that
+      // one of the 'unstarted_scan_ranges' will become the 'next_scan_range_to_start'.
+      if (disk_state->next_scan_range_to_start() == nullptr) {
+        disk_state->ScheduleContext(lock, this, range->disk_id());
+      }
     }
-    // If next_scan_range_to_start is NULL, schedule this RequestContext so that it will
-    // be set. If it's not NULL, this context will be scheduled when GetNextRange() is
-    // invoked.
-    schedule_context = state.next_scan_range_to_start() == NULL;
   } else {
     DCHECK(range->request_type() == RequestType::WRITE);
-    DCHECK(!schedule_immediately);
+    DCHECK(schedule_mode == ScheduleMode::IMMEDIATELY) << static_cast<int>(schedule_mode);
     WriteRange* write_range = static_cast<WriteRange*>(range);
-    state.unstarted_write_ranges()->Enqueue(write_range);
+    disk_state->unstarted_write_ranges()->Enqueue(write_range);
 
-    // ScheduleContext() has no effect if the context is already scheduled,
-    // so this is safe.
-    schedule_context = true;
+    // Ensure that the context is scheduled so that the write range gets picked up.
+    // ScheduleContext() has no effect if already scheduled, so this is safe to do always.
+    disk_state->ScheduleContext(lock, this, range->disk_id());
   }
+  ++disk_state->num_remaining_ranges();
+}
+
+void RequestContext::AddActiveScanRangeLocked(
+    const unique_lock<mutex>& lock, ScanRange* range) {
+  DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
+  DCHECK(state_ == Active);
+  active_scan_ranges_.insert(range);
+}
+
+void RequestContext::RemoveActiveScanRange(ScanRange* range) {
+  unique_lock<mutex> lock(lock_);
+  RemoveActiveScanRangeLocked(lock, range);
+}
 
-  if (schedule_context) state.ScheduleContext(lock, this, range->disk_id());
-  ++state.num_remaining_ranges();
+void RequestContext::RemoveActiveScanRangeLocked(
+    const unique_lock<mutex>& lock, ScanRange* range) {
+  DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
+  active_scan_ranges_.erase(range);
 }
 
 RequestContext::RequestContext(
@@ -235,12 +212,9 @@ string RequestContext::DebugString() const {
   if (state_ == RequestContext::Cancelled) ss << "Cancelled";
   if (state_ == RequestContext::Active) ss << "Active";
   if (state_ != RequestContext::Inactive) {
-    ss << " #ready_buffers=" << num_ready_buffers_.Load()
-       << " #used_buffers=" << num_used_buffers_.Load()
-       << " #num_buffers_in_reader=" << num_buffers_in_reader_.Load()
-       << " #finished_scan_ranges=" << num_finished_ranges_.Load()
-       << " #disk_with_ranges=" << num_disks_with_ranges_
-       << " #disks=" << num_disks_with_ranges_;
+    ss << " #disk_with_ranges=" << num_disks_with_ranges_
+       << " #disks=" << num_disks_with_ranges_
+       << " #active scan ranges=" << active_scan_ranges_.size();
     for (int i = 0; i < disk_states_.size(); ++i) {
       ss << endl << "   " << i << ": "
          << "is_on_queue=" << disk_states_[i].is_on_queue()
@@ -263,16 +237,6 @@ bool RequestContext::Validate() const {
     return false;
   }
 
-  if (num_used_buffers_.Load() < 0) {
-    LOG(WARNING) << "num_used_buffers_ < 0: #used=" << num_used_buffers_.Load();
-    return false;
-  }
-
-  if (num_ready_buffers_.Load() < 0) {
-    LOG(WARNING) << "num_ready_buffers_ < 0: #used=" << num_ready_buffers_.Load();
-    return false;
-  }
-
   int total_unstarted_ranges = 0;
   for (int i = 0; i < disk_states_.size(); ++i) {
     const PerDiskState& state = disk_states_[i];
@@ -350,8 +314,8 @@ bool RequestContext::Validate() const {
       LOG(WARNING) << "Reader cancelled but has ready to start ranges.";
       return false;
     }
-    if (!blocked_ranges_.empty()) {
-      LOG(WARNING) << "Reader cancelled but has blocked ranges.";
+    if (!active_scan_ranges_.empty()) {
+      LOG(WARNING) << "Reader cancelled but has active ranges.";
       return false;
     }
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/request-context.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-context.h b/be/src/runtime/io/request-context.h
index 737a16c..3aea2bc 100644
--- a/be/src/runtime/io/request-context.h
+++ b/be/src/runtime/io/request-context.h
@@ -23,6 +23,11 @@
 
 namespace impala {
 namespace io {
+
+// Mode argument for AddRangeToDisk().
+enum class ScheduleMode {
+  IMMEDIATELY, UPON_GETNEXT, BY_CALLER
+};
 /// A request context is used to group together I/O requests belonging to a client of the
 /// I/O manager for management and scheduling.
 ///
@@ -32,53 +37,59 @@ namespace io {
 /// maintains state across all disks as well as per disk state.
 /// The unit for an IO request is a RequestRange, which may be a ScanRange or a
 /// WriteRange.
-/// A scan range for the reader is on one of five states:
-/// 1) PerDiskState's unstarted_ranges: This range has only been queued
+/// A scan range for the reader is on one of six states:
+/// 1) PerDiskState's 'unstarted_scan_ranges_': This range has only been queued
 ///    and nothing has been read from it.
-/// 2) RequestContext's ready_to_start_ranges_: This range is about to be started.
-///    As soon as the reader picks it up, it will move to the in_flight_ranges
+/// 2) RequestContext's 'ready_to_start_ranges_': This range is about to be started.
+///    As soon as the reader picks it up, it will move to the 'in_flight_ranges_'
 ///    queue.
-/// 3) PerDiskState's in_flight_ranges: This range is being processed and will
+/// 3) PerDiskState's 'in_flight_ranges_': This range is being processed and will
 ///    be read from the next time a disk thread picks it up in GetNextRequestRange()
-/// 4) ScanRange's outgoing ready buffers is full. We can't read for this range
-///    anymore. We need the caller to pull a buffer off which will put this in
-///    the in_flight_ranges queue. These ranges are in the RequestContext's
-///    blocked_ranges_ queue.
-/// 5) ScanRange is cached and in the cached_ranges_ queue.
-//
-/// If the scan range is read and does not get blocked on the outgoing queue, the
+/// 4) The ScanRange is blocked waiting for buffers because it does not have any unused
+///    buffers to read data into. It is unblocked when a client adds new buffers via
+///    AllocateBuffersForRange() or returns existing buffers via ReturnBuffer().
+///    ScanRanges in this state are identified by 'blocked_on_buffer_' == true.
+/// 5) ScanRange is cached and in the 'cached_ranges_' queue.
+/// 6) Inactive - either all the data for the range was returned or the range was
+///    cancelled. I.e. ScanRange::eosr_ is true or ScanRange::cancel_status_ != OK.
+///
+/// If the scan range is read and does not get blocked waiting for buffers, the
 /// transitions are: 1 -> 2 -> 3.
 /// If the scan range does get blocked, the transitions are
 /// 1 -> 2 -> 3 -> (4 -> 3)*
-//
-/// In the case of a cached scan range, the range is immediately put in cached_ranges_.
+///
+/// In the case of a cached scan range, the range is immediately put in 'cached_ranges_'.
 /// When the caller asks for the next range to process, we first pull ranges from
-/// the cache_ranges_ queue. If the range was cached, the range is removed and
+/// the 'cache_ranges_' queue. If the range was cached, the range is removed and
 /// done (ranges are either entirely cached or not at all). If the cached read attempt
 /// fails, we put the range in state 1.
-//
-/// A write range for a context may be in one of two lists:
-/// 1) unstarted_write_ranges_ : Ranges that have been queued but not processed.
-/// 2) in_flight_ranges_: The write range is ready to be processed by the next disk thread
-///    that picks it up in GetNextRequestRange().
-//
+///
+/// All scan ranges in states 1-5 are tracked in 'active_scan_ranges_' so that they can be
+/// cancelled when the RequestContext is cancelled. Scan ranges are removed from
+/// 'active_scan_ranges_' during their transition to state 6.
+///
+/// A write range for a context may be in one of two queues:
+/// 1) 'unstarted_write_ranges_': Ranges that have been queued but not processed.
+/// 2) 'in_flight_ranges_': The write range is ready to be processed by the next disk
+///    thread that picks it up in GetNextRequestRange().
+///
 /// AddWriteRange() adds WriteRanges for a disk.
 /// It is the responsibility of the client to pin the data to be written via a WriteRange
 /// in memory. After a WriteRange has been written, a callback is invoked to inform the
 /// client that the write has completed.
-//
+///
 /// An important assumption is that write does not exceed the maximum read size and that
 /// the entire range is written when the write request is handled. (In other words, writes
 /// are not broken up.)
-//
+///
 /// When a RequestContext is processed by a disk thread in GetNextRequestRange(),
 /// a write range is always removed from the list of unstarted write ranges and appended
 /// to the in_flight_ranges_ queue. This is done to alternate reads and writes - a read
-/// that is scheduled (by calling GetNextRange()) is always followed by a write (if one
-/// exists).  And since at most one WriteRange can be present in in_flight_ranges_ at any
-/// time (once a write range is returned from GetNetxRequestRange() it is completed an
-/// not re-enqueued), a scan range scheduled via a call to GetNextRange() can be queued up
-/// behind at most one write range.
+/// that is scheduled (by calling GetNextUnstartedRange()) is always followed by a write
+/// (if one exists). And since at most one WriteRange can be present in in_flight_ranges_
+/// at any time (once a write range is returned from GetNetxRequestRange() it is completed
+/// and not re-enqueued), a scan range scheduled via a call to GetNextUnstartedRange() can
+/// be queued up behind at most one write range.
 class RequestContext {
  public:
   ~RequestContext() {
@@ -97,7 +108,6 @@ class RequestContext {
     return state_ == Cancelled;
   }
 
-  int64_t queue_size() const { return num_ready_buffers_.Load(); }
   int64_t bytes_read_local() const { return bytes_read_local_.Load(); }
   int64_t bytes_read_short_circuit() const { return bytes_read_short_circuit_.Load(); }
   int64_t bytes_read_dn_cache() const { return bytes_read_dn_cache_.Load(); }
@@ -150,13 +160,6 @@ class RequestContext {
 
   RequestContext(DiskIoMgr* parent, int num_disks, MemTracker* tracker);
 
-  /// Allocates a buffer to read into with size between
-  /// max('buffer_size', 'min_buffer_size_') and 'max_buffer_size_'.
-  /// Does not acquire 'lock_'.
-  /// TODO: allocate using the buffer pool client associated with this reader.
-  Status AllocBuffer(ScanRange* range, int64_t buffer_size,
-      std::unique_ptr<BufferDescriptor>* buffer);
-
   /// Cleans up a buffer. If the buffer was allocated with AllocBuffer(), frees the buffer
   /// memory and release the consumption to the client MemTracker. Otherwise (e.g. a
   /// client or HDFS cache buffer), just prepares the descriptor to be destroyed.
@@ -184,11 +187,11 @@ class RequestContext {
 
   /// Adds range to in_flight_ranges, scheduling this reader on the disk threads
   /// if necessary.
-  /// 'lock_' must be held via 'lock'
+  /// 'lock_' must be held via 'lock'. Only valid to call if this context is active.
   void ScheduleScanRange(const boost::unique_lock<boost::mutex>& lock, ScanRange* range) {
     DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
     DCHECK_EQ(state_, Active);
-    DCHECK(range != NULL);
+    DCHECK(range != nullptr);
     RequestContext::PerDiskState& state = disk_states_[range->disk_id()];
     state.in_flight_ranges()->Enqueue(range);
     state.ScheduleContext(lock, this, range->disk_id());
@@ -198,11 +201,34 @@ class RequestContext {
   /// and mark the context as inactive, after which it cannot be used.
   void CancelAndMarkInactive();
 
-  /// Adds request range to disk queue for this request context. Currently,
-  /// schedule_immediately must be false is RequestRange is a write range.
-  /// Caller must hold 'lock_' via 'lock'.
-  void AddRequestRange(const boost::unique_lock<boost::mutex>& lock,
-      RequestRange* range, bool schedule_immediately);
+  /// Adds a request range to the appropriate disk state. 'schedule_mode' controls which
+  /// queue the range is placed in. This RequestContext is scheduled on the disk state
+  /// if required by 'schedule_mode'.
+  ///
+  /// Write ranges must always have 'schedule_mode' IMMEDIATELY and are added to the
+  /// 'unstarted_write_ranges_' queue, from which they will be asynchronously moved to the
+  /// 'in_flight_ranges_' queue.
+  ///
+  /// Scan ranges can have different 'schedule_mode' values. If IMMEDIATELY, the range is
+  /// immediately added to the 'in_flight_ranges_' queue where it will be processed
+  /// asynchronously by disk threads. If UPON_GETNEXT, the range is added to the
+  /// 'unstarted_ranges_' queue, from which it can be returned to a client by
+  /// DiskIoMgr::GetNextUnstartedRange(). If BY_CALLER, the scan range is not added to
+  /// any queues. The range will be scheduled later as a separate step, e.g. when it is
+  /// unblocked by adding buffers to it. Caller must hold 'lock_' via 'lock'.
+  void AddRangeToDisk(const boost::unique_lock<boost::mutex>& lock, RequestRange* range,
+      ScheduleMode schedule_mode);
+
+  /// Adds an active range to 'active_scan_ranges_'
+  void AddActiveScanRangeLocked(
+      const boost::unique_lock<boost::mutex>& lock, ScanRange* range);
+
+  /// Removes the range from 'active_scan_ranges_'. Called by ScanRange after eos or
+  /// cancellation. If calling the Locked version, the caller must hold
+  /// 'lock_'. Otherwise the function will acquire 'lock_'.
+  void RemoveActiveScanRange(ScanRange* range);
+  void RemoveActiveScanRangeLocked(
+      const boost::unique_lock<boost::mutex>& lock, ScanRange* range);
 
   /// Validates invariants of reader.  Reader lock must be taken beforehand.
   bool Validate() const;
@@ -243,13 +269,6 @@ class RequestContext {
   /// Total number of bytes from remote reads that were expected to be local.
   AtomicInt64 unexpected_remote_bytes_{0};
 
-  /// The number of buffers that have been returned to the reader (via GetNext()) that the
-  /// reader has not returned. Only included for debugging and diagnostics.
-  AtomicInt32 num_buffers_in_reader_{0};
-
-  /// The number of scan ranges that have been completed for this reader.
-  AtomicInt32 num_finished_ranges_{0};
-
   /// The number of scan ranges that required a remote read, updated at the end of each
   /// range scan. Only used for diagnostics.
   AtomicInt32 num_remote_ranges_{0};
@@ -264,17 +283,6 @@ class RequestContext {
   /// Total number of file handle opens where the file handle was not in the cache
   AtomicInt32 cached_file_handles_miss_count_{0};
 
-  /// The number of buffers that are being used for this reader. This is the sum
-  /// of all buffers in ScanRange queues and buffers currently being read into (i.e. about
-  /// to be queued). This includes both IOMgr-allocated buffers and client-provided
-  /// buffers.
-  AtomicInt32 num_used_buffers_{0};
-
-  /// The total number of ready buffers across all ranges.  Ready buffers are buffers
-  /// that have been read from disk but not retrieved by the caller.
-  /// This is the sum of all queued buffers in all ranges for this reader context.
-  AtomicInt32 num_ready_buffers_{0};
-
   /// All fields below are accessed by multiple threads and the lock needs to be
   /// taken before accessing them. Must be acquired before ScanRange::lock_ if both
   /// are held simultaneously.
@@ -283,6 +291,17 @@ class RequestContext {
   /// Current state of the reader
   State state_ = Active;
 
+  /// Scan ranges that have been added to the IO mgr for this context. Ranges can only
+  /// be added when 'state_' is Active. When this context is cancelled, Cancel() is
+  /// called for all the active ranges. If a client attempts to add a range while
+  /// 'state_' is Cancelled, the range is not added to this list and Status::CANCELLED
+  /// is returned to the client. This ensures that all active ranges are cancelled as a
+  /// result of RequestContext cancellation.
+  /// Ranges can be cancelled or hit eos non-atomically with their removal from this set,
+  /// so eos or cancelled ranges may be temporarily present here. Cancelling these ranges
+  /// a second time or cancelling after eos is safe and has no effect.
+  boost::unordered_set<ScanRange*> active_scan_ranges_;
+
   /// The number of disks with scan ranges remaining (always equal to the sum of
   /// disks with ranges).
   int num_disks_with_ranges_ = 0;
@@ -293,18 +312,15 @@ class RequestContext {
   InternalList<ScanRange> cached_ranges_;
 
   /// A list of ranges that should be returned in subsequent calls to
-  /// GetNextRange.
+  /// GetNextUnstartedRange().
   /// There is a trade-off with when to populate this list.  Populating it on
-  /// demand means consumers need to wait (happens in DiskIoMgr::GetNextRange()).
+  /// demand means consumers need to wait (happens in DiskIoMgr::GetNextUnstartedRange()).
   /// Populating it preemptively means we make worse scheduling decisions.
   /// We currently populate one range per disk.
   /// TODO: think about this some more.
   InternalList<ScanRange> ready_to_start_ranges_;
   ConditionVariable ready_to_start_ranges_cv_; // used with lock_
 
-  /// Ranges that are blocked due to back pressure on outgoing buffers.
-  InternalList<ScanRange> blocked_ranges_;
-
   /// Condition variable for UnregisterContext() to wait for all disks to complete
   ConditionVariable disks_complete_cond_var_;
 
@@ -429,7 +445,7 @@ class RequestContext {
 
     /// Queue of pending IO requests for this disk in the order that they will be
     /// processed. A ScanRange is added to this queue when it is returned in
-    /// GetNextRange(), or when it is added with schedule_immediately = true.
+    /// GetNextUnstartedRange(), or when it is added with schedule_mode == IMMEDIATELY.
     /// A WriteRange is added to this queue from unstarted_write_ranges_ for each
     /// invocation of GetNextRequestRange() in WorkLoop().
     /// The size of this queue is always less than or equal to num_remaining_ranges.
@@ -458,7 +474,8 @@ class RequestContext {
     /// unstarted_read_ranges_ and unstarted_write_ranges_ to alternate between reads
     /// and writes. (Otherwise, since next_scan_range_to_start is set
     /// in GetNextRequestRange() whenever it is null, repeated calls to
-    /// GetNextRequestRange() and GetNextRange() may result in only reads being processed)
+    /// GetNextRequestRange() and GetNextUnstartedRange() may result in only reads being
+    /// processed)
     InternalQueue<WriteRange> unstarted_write_ranges_;
   };
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/request-ranges.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-ranges.h b/be/src/runtime/io/request-ranges.h
index ab0810a..041cb9d 100644
--- a/be/src/runtime/io/request-ranges.h
+++ b/be/src/runtime/io/request-ranges.h
@@ -26,8 +26,10 @@
 #include "common/atomic.h"
 #include "common/hdfs.h"
 #include "common/status.h"
+#include "runtime/bufferpool/buffer-pool.h"
 #include "util/condition-variable.h"
 #include "util/internal-queue.h"
+#include "util/mem-range.h"
 
 namespace impala {
 class MemTracker;
@@ -202,11 +204,11 @@ class ScanRange : public RequestRange {
 
   /// Resets this scan range object with the scan range description. The scan range
   /// is for bytes [offset, offset + len) in 'file' on 'fs' (which is nullptr for the
-  /// local filesystem). The scan range must fall within the file bounds (offset >= 0
-  /// and offset + len <= file_length). 'disk_id' is the disk queue to add the range
-  /// to. If 'expected_local' is true, a warning is generated if the read did not
-  /// come from a local disk. 'buffer_opts' specifies buffer management options -
-  /// see the DiskIoMgr class comment and the BufferOpts comments for details.
+  /// local filesystem). The scan range must be non-empty and fall within the file bounds
+  /// (len > 0 and offset >= 0 and offset + len <= file_length). 'disk_id' is the disk
+  /// queue to add the range to. If 'expected_local' is true, a warning is generated if
+  /// the read did not come from a local disk. 'buffer_opts' specifies buffer management
+  /// options - see the DiskIoMgr class comment and the BufferOpts comments for details.
   /// 'meta_data' is an arbitrary client-provided pointer for any auxiliary data.
   void Reset(hdfsFS fs, const char* file, int64_t len, int64_t offset, int disk_id,
       bool expected_local, const BufferOpts& buffer_opts, void* meta_data = nullptr);
@@ -248,12 +250,11 @@ class ScanRange : public RequestRange {
   /// Initialize internal fields
   void InitInternal(DiskIoMgr* io_mgr, RequestContext* reader);
 
-  /// Enqueues a buffer for this range. This does not block.
-  /// Returns true if this scan range has hit the queue capacity, false otherwise.
+  /// Enqueues a ready buffer with valid data for this range. This does not block.
   /// The caller passes ownership of buffer to the scan range and it is not
   /// valid to access buffer after this call. The reader lock must be held by the
-  /// caller.
-  bool EnqueueBuffer(const boost::unique_lock<boost::mutex>& reader_lock,
+  /// caller. Returns false if the scan range was cancelled.
+  bool EnqueueReadyBuffer(const boost::unique_lock<boost::mutex>& reader_lock,
       std::unique_ptr<BufferDescriptor> buffer);
 
   /// Validates the internal state of this range. lock_ must be taken
@@ -304,16 +305,51 @@ class ScanRange : public RequestRange {
   Status ReadFromCache(const boost::unique_lock<boost::mutex>& reader_lock,
       bool* read_succeeded) WARN_UNUSED_RESULT;
 
+  /// Add buffers for the range to read data into and schedule the range if blocked.
+  /// If 'returned' is true, the buffers returned from GetNext() that are being recycled
+  /// via ReturnBuffer(). Otherwise the buffers are newly allocated buffers to be added.
+  void AddUnusedBuffers(
+      std::vector<std::unique_ptr<BufferDescriptor>>&& buffers, bool returned);
+
+  /// Remove a buffer from 'unused_iomgr_buffers_' and update
+  /// 'unused_iomgr_buffer_bytes_'. If 'unused_iomgr_buffers_' is empty, return NULL.
+  /// 'lock_' must be held by the caller via 'scan_range_lock'.
+  std::unique_ptr<BufferDescriptor> GetUnusedBuffer(
+      const boost::unique_lock<boost::mutex>& scan_range_lock);
+
+  /// Get the next buffer for this scan range for a disk thread to read into. Returns
+  /// the new buffer if successful.  If no buffers are available, marks the range
+  /// as blocked and returns nullptr. Called must not hold 'lock_'.
+  std::unique_ptr<BufferDescriptor> GetNextUnusedBufferForRange();
+
   /// Cleans up a buffer that was not returned to the client.
   /// Either ReturnBuffer() or CleanUpBuffer() is called for every BufferDescriptor.
-  /// This function will acquire 'lock_' and may acquire 'hdfs_lock_'.
-  void CleanUpBuffer(std::unique_ptr<BufferDescriptor> buffer);
-
-  /// Same as CleanUpBuffer() except the caller must already hold 'lock_' via
-  /// 'scan_range_lock'.
-  void CleanUpBufferLocked(const boost::unique_lock<boost::mutex>& scan_range_lock,
+  /// The caller must hold 'lock_' via 'scan_range_lock'.
+  /// This function may acquire 'hdfs_lock_'
+  void CleanUpBuffer(const boost::unique_lock<boost::mutex>& scan_range_lock,
       std::unique_ptr<BufferDescriptor> buffer);
 
+  /// Same as CleanUpBuffer() except cleans up multiple buffers and caller must not
+  /// hold 'lock_'.
+  void CleanUpBuffers(std::vector<std::unique_ptr<BufferDescriptor>>&& buffers);
+
+  /// Clean up all buffers in 'unused_iomgr_buffers_'. Only valid to call when the scan
+  /// range is cancelled or at eos. The caller must hold 'lock_' via 'scan_range_lock'.
+  void CleanUpUnusedBuffers(const boost::unique_lock<boost::mutex>& scan_range_lock);
+
+  /// Same as Cancel() except reader_->lock must be held by the caller.
+  void CancelFromReader(const boost::unique_lock<boost::mutex>& reader_lock,
+      const Status& status);
+
+  /// Same as Cancel() except doesn't remove the scan range from
+  /// reader_->active_scan_ranges_. This is invoked by RequestContext::Cancel(),
+  /// which removes the range itself to avoid invalidating its active_scan_ranges_
+  /// iterator.
+  void CancelInternal(const Status& status);
+
+  /// Marks the scan range as blocked waiting for a buffer. Caller must not hold 'lock_'.
+  void SetBlockedOnBuffer();
+
   /// Returns true if no more buffers will be returned to clients in the future,
   /// either because of hitting eosr or cancellation.
   bool all_buffers_returned(const boost::unique_lock<boost::mutex>& lock) const {
@@ -386,6 +422,10 @@ class ScanRange : public RequestRange {
     struct hadoopRzBuffer* cached_buffer_ = nullptr;
   };
 
+  /// The number of buffers that have been returned to a client via GetNext() that have
+  /// not yet been returned with ReturnBuffer().
+  AtomicInt32 num_buffers_in_reader_{0};
+
   /// Lock protecting fields below.
   /// This lock should not be taken during Open()/Read()/Close().
   /// If RequestContext::lock_ and this lock need to be held simultaneously,
@@ -395,18 +435,30 @@ class ScanRange : public RequestRange {
   /// Number of bytes read so far for this scan range
   int bytes_read_;
 
-  /// The number of buffers that have been returned to a client via GetNext() that have
-  /// not yet been returned with ReturnBuffer().
-  int num_buffers_in_reader_ = 0;
+  /// Buffers to read into, used if the 'external_buffer_tag_' is NO_BUFFER. These are
+  /// initially populated when the client calls AllocateBuffersForRange() and
+  /// and are used to read scanned data into. Buffers are taken from this vector for
+  /// every read and added back, if needed, when the client calls ReturnBuffer().
+  std::vector<std::unique_ptr<BufferDescriptor>> unused_iomgr_buffers_;
+
+  /// Total number of bytes of buffers in 'unused_iomgr_buffers_'.
+  int64_t unused_iomgr_buffer_bytes_ = 0;
+
+  /// Number of bytes of buffers returned from GetNextUnusedBufferForRange(). Used to
+  /// infer how many bytes of buffers need to be held onto to read the rest of the scan
+  /// range.
+  int64_t iomgr_buffer_bytes_returned_ = 0;
 
   /// If true, the last buffer for this scan range has been queued.
   /// If this is true and 'ready_buffers_' is empty, then no more buffers will be
   /// returned to the caller by this scan range.
   bool eosr_queued_ = false;
 
-  /// If true, this scan range has been removed from the reader's in_flight_ranges
-  /// queue because the ready_buffers_ queue is full.
-  bool blocked_on_queue_ = false;
+  /// If true, this scan range is not scheduled because a buffer is not available for
+  /// the next I/O in the range. This can happen when the scan range is initially created
+  /// or if the buffers added to the range have all been filled with data an not yet
+  /// returned.
+  bool blocked_on_buffer_ = false;
 
   /// IO buffers that are queued for this scan range. When Cancel() is called
   /// this is drained by the cancelling thread. I.e. this is always empty if

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/io/scan-range.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/scan-range.cc b/be/src/runtime/io/scan-range.cc
index 1ffba00..0663a2b 100644
--- a/be/src/runtime/io/scan-range.cc
+++ b/be/src/runtime/io/scan-range.cc
@@ -42,7 +42,7 @@ DEFINE_int64(adls_read_chunk_size, 128 * 1024, "The maximum read chunk size to u
 // any time and only one thread will remove from the queue. This is to guarantee
 // that buffers are queued and read in file order.
 
-bool ScanRange::EnqueueBuffer(
+bool ScanRange::EnqueueReadyBuffer(
     const unique_lock<mutex>& reader_lock, unique_ptr<BufferDescriptor> buffer) {
   DCHECK(reader_lock.mutex() == &reader_->lock_ && reader_lock.owns_lock());
   DCHECK(buffer->buffer_ != nullptr) << "Cannot enqueue freed buffer";
@@ -52,19 +52,17 @@ bool ScanRange::EnqueueBuffer(
     DCHECK(!eosr_queued_);
     if (!cancel_status_.ok()) {
       // This range has been cancelled, no need to enqueue the buffer.
-      reader_->num_used_buffers_.Add(-1);
-      CleanUpBufferLocked(scan_range_lock, move(buffer));
+      CleanUpBuffer(scan_range_lock, move(buffer));
       return false;
     }
-    reader_->num_ready_buffers_.Add(1);
+    // Clean up any surplus buffers. E.g. we may have allocated too many if the file was
+    // shorter than expected.
+    if (buffer->eosr()) CleanUpUnusedBuffers(scan_range_lock);
     eosr_queued_ = buffer->eosr();
     ready_buffers_.emplace_back(move(buffer));
-
-    DCHECK_LE(ready_buffers_.size(), DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT);
-    blocked_on_queue_ = ready_buffers_.size() == DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT;
   }
   buffer_ready_cv_.NotifyOne();
-  return blocked_on_queue_;
+  return true;
 }
 
 Status ScanRange::GetNext(unique_ptr<BufferDescriptor>* buffer) {
@@ -84,64 +82,97 @@ Status ScanRange::GetNext(unique_ptr<BufferDescriptor>* buffer) {
 
     // Remove the first ready buffer from the queue and return it
     DCHECK(!ready_buffers_.empty());
-    DCHECK_LE(ready_buffers_.size(), DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT);
     *buffer = move(ready_buffers_.front());
     ready_buffers_.pop_front();
     eosr = (*buffer)->eosr();
+    DCHECK(!eosr || unused_iomgr_buffers_.empty()) << DebugString();
   }
 
-  // Update tracking counters. The buffer has now moved from the IoMgr to the
-  // caller.
-  reader_->num_ready_buffers_.Add(-1);
-  reader_->num_used_buffers_.Add(-1);
-  if (eosr) reader_->num_finished_ranges_.Add(1);
-
-  unique_lock<mutex> reader_lock(reader_->lock_);
+  // Update tracking counters. The buffer has now moved from the IoMgr to the caller.
+  if (eosr) reader_->RemoveActiveScanRange(this);
+  num_buffers_in_reader_.Add(1);
+  return Status::OK();
+}
 
-  DCHECK(reader_->Validate()) << endl << reader_->DebugString();
-  if (reader_->state_ == RequestContext::Cancelled) {
-    reader_->blocked_ranges_.Remove(this);
-    Cancel(Status::CANCELLED);
-    CleanUpBuffer(move(*buffer));
-    return Status::CANCELLED;
-  }
+void ScanRange::ReturnBuffer(unique_ptr<BufferDescriptor> buffer_desc) {
+  vector<unique_ptr<BufferDescriptor>> buffers;
+  buffers.emplace_back(move(buffer_desc));
+  AddUnusedBuffers(move(buffers), true);
+}
 
-  // At this point success is guaranteed so increment counters for returned buffers.
-  reader_->num_buffers_in_reader_.Add(1);
+void ScanRange::AddUnusedBuffers(vector<unique_ptr<BufferDescriptor>>&& buffers,
+      bool returned) {
+  DCHECK_GT(buffers.size(), 0);
+  /// Keep track of whether the range was unblocked in this function. If so, we need
+  /// to schedule it so it resumes progress.
+  bool unblocked = false;
   {
-    // Check to see if we can re-schedule a blocked range. Note that EnqueueBuffer()
-    // may have been called after we released 'lock_' above so we need to re-check
-    // whether the queue is full.
     unique_lock<mutex> scan_range_lock(lock_);
-    ++num_buffers_in_reader_;
-    if (blocked_on_queue_
-        && ready_buffers_.size() < DiskIoMgr::SCAN_RANGE_READY_BUFFER_LIMIT
-        && !eosr_queued_) {
-      blocked_on_queue_ = false;
-      // This scan range was blocked and is no longer, add it to the reader
-      // queue again.
-      reader_->blocked_ranges_.Remove(this);
+    if (returned) {
+      // Buffers were in reader but now aren't.
+      num_buffers_in_reader_.Add(-buffers.size());
+    }
+
+    for (unique_ptr<BufferDescriptor>& buffer : buffers) {
+      // We should not hold onto the buffers in the following cases:
+      // 1. the scan range is using external buffers, e.g. cached buffers.
+      // 2. the scan range is cancelled
+      // 3. the scan range already hit eosr
+      // 4. we already have enough buffers to read the remainder of the scan range.
+      if (external_buffer_tag_ != ExternalBufferTag::NO_BUFFER
+          || !cancel_status_.ok()
+          || eosr_queued_
+          || unused_iomgr_buffer_bytes_ >= len_ - iomgr_buffer_bytes_returned_) {
+        CleanUpBuffer(scan_range_lock, move(buffer));
+      } else {
+        unused_iomgr_buffer_bytes_ += buffer->buffer_len();
+        unused_iomgr_buffers_.emplace_back(move(buffer));
+        if (blocked_on_buffer_) {
+          blocked_on_buffer_ = false;
+          unblocked = true;
+        }
+      }
+    }
+  }
+  // Must drop the ScanRange lock before acquiring the RequestContext lock.
+  if (unblocked) {
+    unique_lock<mutex> reader_lock(reader_->lock_);
+    // Reader may have been cancelled after we dropped 'scan_range_lock' above.
+    if (reader_->state_ == RequestContext::Cancelled) {
+      DCHECK(!cancel_status_.ok());
+    } else {
       reader_->ScheduleScanRange(reader_lock, this);
     }
   }
-  return Status::OK();
 }
 
-void ScanRange::ReturnBuffer(unique_ptr<BufferDescriptor> buffer_desc) {
-  reader_->num_buffers_in_reader_.Add(-1);
-  {
-    unique_lock<mutex> scan_range_lock(lock_);
-    --num_buffers_in_reader_;
-    CleanUpBufferLocked(scan_range_lock, move(buffer_desc));
+unique_ptr<BufferDescriptor> ScanRange::GetUnusedBuffer(
+    const unique_lock<mutex>& scan_range_lock) {
+  DCHECK(scan_range_lock.mutex() == &lock_ && scan_range_lock.owns_lock());
+  if (unused_iomgr_buffers_.empty()) return nullptr;
+  unique_ptr<BufferDescriptor> result = move(unused_iomgr_buffers_.back());
+  unused_iomgr_buffers_.pop_back();
+  unused_iomgr_buffer_bytes_ -= result->buffer_len();
+  return result;
+}
+
+unique_ptr<BufferDescriptor> ScanRange::GetNextUnusedBufferForRange() {
+  unique_lock<mutex> lock(lock_);
+  unique_ptr<BufferDescriptor> buffer_desc = GetUnusedBuffer(lock);
+  if (buffer_desc == nullptr) {
+    blocked_on_buffer_ = true;
+  } else {
+    iomgr_buffer_bytes_returned_ += buffer_desc->buffer_len();
   }
+  return buffer_desc;
 }
 
-void ScanRange::CleanUpBuffer(unique_ptr<BufferDescriptor> buffer_desc) {
-  unique_lock<mutex> scan_range_lock(lock_);
-  CleanUpBufferLocked(scan_range_lock, move(buffer_desc));
+void ScanRange::SetBlockedOnBuffer() {
+  unique_lock<mutex> lock(lock_);
+  blocked_on_buffer_ = true;
 }
 
-void ScanRange::CleanUpBufferLocked(
+void ScanRange::CleanUpBuffer(
     const boost::unique_lock<boost::mutex>& scan_range_lock,
     unique_ptr<BufferDescriptor> buffer_desc) {
   DCHECK(scan_range_lock.mutex() == &lock_ && scan_range_lock.owns_lock());
@@ -149,7 +180,7 @@ void ScanRange::CleanUpBufferLocked(
   DCHECK_EQ(this, buffer_desc->scan_range_);
   buffer_desc->reader_->FreeBuffer(buffer_desc.get());
 
-  if (all_buffers_returned(scan_range_lock) && num_buffers_in_reader_ == 0) {
+  if (all_buffers_returned(scan_range_lock) && num_buffers_in_reader_.Load() == 0) {
     // Close the scan range if there are no more buffers in the reader and no more buffers
     // will be returned to readers in future. Close() is idempotent so it is ok to call
     // multiple times during cleanup so long as the range is actually finished.
@@ -157,10 +188,33 @@ void ScanRange::CleanUpBufferLocked(
   }
 }
 
+void ScanRange::CleanUpBuffers(vector<unique_ptr<BufferDescriptor>>&& buffers) {
+  unique_lock<mutex> lock(lock_);
+  for (unique_ptr<BufferDescriptor>& buffer : buffers) CleanUpBuffer(lock, move(buffer));
+}
+
+void ScanRange::CleanUpUnusedBuffers(const unique_lock<mutex>& scan_range_lock) {
+  while (!unused_iomgr_buffers_.empty()) {
+    CleanUpBuffer(scan_range_lock, GetUnusedBuffer(scan_range_lock));
+  }
+}
+
 void ScanRange::Cancel(const Status& status) {
   // Cancelling a range that was never started, ignore.
   if (io_mgr_ == nullptr) return;
+  CancelInternal(status);
+  reader_->RemoveActiveScanRange(this);
+}
+
+void ScanRange::CancelFromReader(const boost::unique_lock<boost::mutex>& reader_lock,
+    const Status& status) {
+  DCHECK(reader_lock.mutex() == &reader_->lock_ && reader_lock.owns_lock());
+  CancelInternal(status);
+  reader_->RemoveActiveScanRangeLocked(reader_lock, this);
+}
 
+void ScanRange::CancelInternal(const Status& status) {
+  DCHECK(io_mgr_ != nullptr);
   DCHECK(!status.ok());
   {
     // Grab both locks to make sure that we don't change 'cancel_status_' while other
@@ -177,12 +231,13 @@ void ScanRange::Cancel(const Status& status) {
 
     /// Clean up 'ready_buffers_' while still holding 'lock_' to prevent other threads
     /// from seeing inconsistent state.
-    reader_->num_used_buffers_.Add(-ready_buffers_.size());
-    reader_->num_ready_buffers_.Add(-ready_buffers_.size());
     while (!ready_buffers_.empty()) {
-      CleanUpBufferLocked(scan_range_lock, move(ready_buffers_.front()));
+      CleanUpBuffer(scan_range_lock, move(ready_buffers_.front()));
       ready_buffers_.pop_front();
     }
+
+    /// Clean up buffers that we don't need any more because we won't read any more data.
+    CleanUpUnusedBuffers(scan_range_lock);
   }
   buffer_ready_cv_.NotifyAll();
 
@@ -197,6 +252,10 @@ string ScanRange::DebugString() const {
      << " len=" << len_ << " bytes_read=" << bytes_read_
      << " cancel_status=" << cancel_status_.GetDetail()
      << " buffer_queue=" << ready_buffers_.size()
+     << " num_buffers_in_readers=" << num_buffers_in_reader_.Load()
+     << " unused_iomgr_buffers=" << unused_iomgr_buffers_.size()
+     << " unused_iomgr_buffer_bytes=" << unused_iomgr_buffer_bytes_
+     << " blocked_on_buffer=" << blocked_on_buffer_
      << " hdfs_file=" << exclusive_hdfs_fh_;
   return ss.str();
 }
@@ -211,6 +270,27 @@ bool ScanRange::Validate() {
     LOG(ERROR) << "Cancelled range should not have queued buffers " << DebugString();
     return false;
   }
+  int64_t unused_iomgr_buffer_bytes = 0;
+  for (auto& buffer : unused_iomgr_buffers_)
+    unused_iomgr_buffer_bytes += buffer->buffer_len();
+  if (unused_iomgr_buffer_bytes != unused_iomgr_buffer_bytes_) {
+    LOG(ERROR) << "unused_iomgr_buffer_bytes_ incorrect actual: "
+               << unused_iomgr_buffer_bytes_
+               << " vs. expected: " << unused_iomgr_buffer_bytes;
+    return false;
+  }
+  bool is_finished = !cancel_status_.ok() || eosr_queued_;
+  if (is_finished && !unused_iomgr_buffers_.empty()) {
+    LOG(ERROR) << "Held onto too many buffers " << unused_iomgr_buffers_.size()
+               << " bytes: " << unused_iomgr_buffer_bytes_
+               << " cancel_status: " << cancel_status_.GetDetail()
+               << " eosr_queued: " << eosr_queued_;
+    return false;
+  }
+  if (!is_finished && blocked_on_buffer_ && !unused_iomgr_buffers_.empty()) {
+    LOG(ERROR) << "Blocked despite having buffers: " << DebugString();
+    return false;
+  }
   return true;
 }
 
@@ -224,7 +304,7 @@ ScanRange::~ScanRange() {
   DCHECK(external_buffer_tag_ != ExternalBufferTag::CACHED_BUFFER)
       << "Cached buffer was not released.";
   DCHECK_EQ(0, ready_buffers_.size());
-  DCHECK_EQ(0, num_buffers_in_reader_);
+  DCHECK_EQ(0, num_buffers_in_reader_.Load());
 }
 
 void ScanRange::Reset(hdfsFS fs, const char* file, int64_t len, int64_t offset,
@@ -268,9 +348,11 @@ void ScanRange::InitInternal(DiskIoMgr* io_mgr, RequestContext* reader) {
   local_file_ = nullptr;
   exclusive_hdfs_fh_ = nullptr;
   bytes_read_ = 0;
+  unused_iomgr_buffer_bytes_ = 0;
+  iomgr_buffer_bytes_returned_ = 0;
   cancel_status_ = Status::OK();
-  eosr_queued_= false;
-  blocked_on_queue_ = false;
+  eosr_queued_ = false;
+  blocked_on_buffer_ = false;
   DCHECK(Validate()) << DebugString();
 }
 
@@ -316,9 +398,7 @@ Status ScanRange::Open(bool use_file_handle_cache) {
           "for file: $1: $2", offset_, file_, GetStrErrMsg()));
     }
   }
-  if (ImpaladMetrics::IO_MGR_NUM_OPEN_FILES != nullptr) {
-    ImpaladMetrics::IO_MGR_NUM_OPEN_FILES->Increment(1L);
-  }
+  ImpaladMetrics::IO_MGR_NUM_OPEN_FILES->Increment(1L);
   return Status::OK();
 }
 
@@ -370,9 +450,7 @@ void ScanRange::Close() {
     local_file_ = nullptr;
     closed_file = true;
   }
-  if (closed_file && ImpaladMetrics::IO_MGR_NUM_OPEN_FILES != nullptr) {
-    ImpaladMetrics::IO_MGR_NUM_OPEN_FILES->Increment(-1L);
-  }
+  if (closed_file) ImpaladMetrics::IO_MGR_NUM_OPEN_FILES->Increment(-1L);
 }
 
 int64_t ScanRange::MaxReadChunkSize() const {
@@ -580,12 +658,9 @@ Status ScanRange::ReadFromCache(
   desc->scan_range_offset_ = 0;
   desc->eosr_ = true;
   bytes_read_ = bytes_read;
-  EnqueueBuffer(reader_lock, move(desc));
-  if (reader_->bytes_read_counter_ != nullptr) {
-    COUNTER_ADD(reader_->bytes_read_counter_, bytes_read);
-  }
+  EnqueueReadyBuffer(reader_lock, move(desc));
+  COUNTER_ADD_IF_NOT_NULL(reader_->bytes_read_counter_, bytes_read);
   *read_succeeded = true;
-  reader_->num_used_buffers_.Add(1);
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/tmp-file-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr.cc b/be/src/runtime/tmp-file-mgr.cc
index 7b00179..3a69c33 100644
--- a/be/src/runtime/tmp-file-mgr.cc
+++ b/be/src/runtime/tmp-file-mgr.cc
@@ -396,7 +396,10 @@ Status TmpFileMgr::FileGroup::ReadAsync(WriteHandle* handle, MemRange buffer) {
       BufferOpts::ReadInto(buffer.data(), buffer.len()));
   read_counter_->Add(1);
   bytes_read_counter_->Add(buffer.len());
-  RETURN_IF_ERROR(io_mgr_->AddScanRange(io_ctx_.get(), handle->read_range_, true));
+  bool needs_buffers;
+  RETURN_IF_ERROR(io_mgr_->StartScanRange(
+      io_ctx_.get(), handle->read_range_, &needs_buffers));
+  DCHECK(!needs_buffers) << "Already provided a buffer";
   return Status::OK();
 }
 
@@ -522,11 +525,20 @@ TmpFileMgr::WriteHandle::WriteHandle(
     is_cancelled_(false),
     write_in_flight_(false) {}
 
+TmpFileMgr::WriteHandle::~WriteHandle() {
+  DCHECK(!write_in_flight_);
+  DCHECK(read_range_ == nullptr);
+}
+
 string TmpFileMgr::WriteHandle::TmpFilePath() const {
   if (file_ == nullptr) return "";
   return file_->path();
 }
 
+int64_t TmpFileMgr::WriteHandle::len() const {
+  return write_range_->len();
+}
+
 Status TmpFileMgr::WriteHandle::Write(DiskIoMgr* io_mgr, RequestContext* io_ctx,
     File* file, int64_t offset, MemRange buffer,
     WriteRange::WriteDoneCallback callback) {

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/runtime/tmp-file-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr.h b/be/src/runtime/tmp-file-mgr.h
index 95072ae..55901f4 100644
--- a/be/src/runtime/tmp-file-mgr.h
+++ b/be/src/runtime/tmp-file-mgr.h
@@ -28,7 +28,6 @@
 #include "common/object-pool.h"
 #include "common/status.h"
 #include "gen-cpp/Types_types.h" // for TUniqueId
-#include "runtime/io/request-ranges.h"
 #include "util/collection-metrics.h"
 #include "util/condition-variable.h"
 #include "util/mem-range.h"
@@ -37,6 +36,12 @@
 #include "util/spinlock.h"
 
 namespace impala {
+namespace io {
+  class DiskIoMgr;
+  class RequestContext;
+  class ScanRange;
+  class WriteRange;
+}
 
 /// TmpFileMgr provides an abstraction for management of temporary (a.k.a. scratch) files
 /// on the filesystem and I/O to and from them. TmpFileMgr manages multiple scratch
@@ -84,6 +89,7 @@ class TmpFileMgr {
   /// Needs to be public for TmpFileMgrTest.
   typedef int DeviceId;
 
+  /// Same typedef as io::WriteRange::WriteDoneCallback.
   typedef std::function<void(const Status&)> WriteDoneCallback;
 
   /// Represents a group of temporary files - one per disk with a scratch directory. The
@@ -277,10 +283,7 @@ class TmpFileMgr {
    public:
     /// The write must be destroyed by passing it to FileGroup - destroying it before
     /// the write completes is an error.
-    ~WriteHandle() {
-      DCHECK(!write_in_flight_);
-      DCHECK(read_range_ == nullptr);
-    }
+    ~WriteHandle();
 
     /// Cancel any in-flight read synchronously.
     void CancelRead();
@@ -290,7 +293,7 @@ class TmpFileMgr {
     std::string TmpFilePath() const;
 
     /// The length of the write range in bytes.
-    int64_t len() const { return write_range_->len(); }
+    int64_t len() const;
 
     std::string DebugString();
 
@@ -305,7 +308,7 @@ class TmpFileMgr {
     /// failure and 'is_cancelled_' is set to true on failure.
     Status Write(io::DiskIoMgr* io_mgr, io::RequestContext* io_ctx, File* file,
         int64_t offset, MemRange buffer,
-        io::WriteRange::WriteDoneCallback callback) WARN_UNUSED_RESULT;
+        WriteDoneCallback callback) WARN_UNUSED_RESULT;
 
     /// Retry the write after the initial write failed with an error, instead writing to
     /// 'offset' of 'file'. 'write_in_flight_' must be true before calling.

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/util/bit-util-test.cc
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util-test.cc b/be/src/util/bit-util-test.cc
index 5f8d443..6f70727 100644
--- a/be/src/util/bit-util-test.cc
+++ b/be/src/util/bit-util-test.cc
@@ -258,6 +258,17 @@ TEST(BitUtil, Log2) {
   EXPECT_EQ(BitUtil::Log2CeilingNonZero64(ULLONG_MAX), 64);
 }
 
+TEST(BitUtil, RoundToPowerOfTwo) {
+  EXPECT_EQ(16, BitUtil::RoundUpToPowerOfTwo(9));
+  EXPECT_EQ(16, BitUtil::RoundUpToPowerOfTwo(15));
+  EXPECT_EQ(16, BitUtil::RoundUpToPowerOfTwo(16));
+  EXPECT_EQ(32, BitUtil::RoundUpToPowerOfTwo(17));
+  EXPECT_EQ(8, BitUtil::RoundDownToPowerOfTwo(9));
+  EXPECT_EQ(8, BitUtil::RoundDownToPowerOfTwo(15));
+  EXPECT_EQ(16, BitUtil::RoundDownToPowerOfTwo(16));
+  EXPECT_EQ(16, BitUtil::RoundDownToPowerOfTwo(17));
+}
+
 TEST(BitUtil, RoundUpToPowerOf2) {
   EXPECT_EQ(BitUtil::RoundUpToPowerOf2(7, 8), 8);
   EXPECT_EQ(BitUtil::RoundUpToPowerOf2(8, 8), 8);

http://git-wip-us.apache.org/repos/asf/impala/blob/0b6fab73/be/src/util/bit-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/bit-util.h b/be/src/util/bit-util.h
index 5c9a29b..8a65509 100644
--- a/be/src/util/bit-util.h
+++ b/be/src/util/bit-util.h
@@ -98,6 +98,12 @@ class BitUtil {
     return v;
   }
 
+  /// Returns the largest power of two <= v.
+  static inline int64_t RoundDownToPowerOfTwo(int64_t v) {
+    int64_t v_rounded_up = RoundUpToPowerOfTwo(v);
+    return v_rounded_up == v ? v : v_rounded_up / 2;
+  }
+
   /// Returns 'value' rounded up to the nearest multiple of 'factor' when factor is
   /// a power of two
   static inline int64_t RoundUpToPowerOf2(int64_t value, int64_t factor) {
@@ -105,7 +111,7 @@ class BitUtil {
     return (value + (factor - 1)) & ~(factor - 1);
   }
 
-  static inline int RoundDownToPowerOf2(int value, int factor) {
+  static inline int64_t RoundDownToPowerOf2(int64_t value, int64_t factor) {
     DCHECK((factor > 0) && ((factor & (factor - 1)) == 0));
     return value & ~(factor - 1);
   }


[03/15] impala git commit: IMPALA-4835: Part 3: switch I/O buffers to buffer pool

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
index 261e8fe..003e68f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/resource-requirements.test
@@ -1,24 +1,795 @@
 # Parquet scan
 select * from tpch_parquet.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=72.00MB
+Per-Host Resource Estimates: Memory=80.00MB
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=72.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpch_parquet.lineitem]
+   partitions=1/1 files=3 size=193.71MB
+   stored statistics:
+     table: rows=6001215 size=193.71MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=80.00MB mem-reservation=72.00MB
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=72.00MB
+Per-Host Resource Estimates: Memory=80.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=80.00MB mem-reservation=72.00MB
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.71MB
+   stored statistics:
+     table: rows=6001215 size=193.71MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=80.00MB mem-reservation=72.00MB
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=144.00MB
+Per-Host Resource Estimates: Memory=160.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Resources: mem-estimate=160.00MB mem-reservation=144.00MB
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.71MB
+   stored statistics:
+     table: rows=6001215 size=193.71MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=80.00MB mem-reservation=72.00MB
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# Single column parquet scan - memory reservation is reduced compared to multi-column
+# scan.
+select l_comment from tpch_parquet.lineitem
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=80.00MB
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=8.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpch_parquet.lineitem]
+   partitions=1/1 files=3 size=193.71MB
+   stored statistics:
+     table: rows=6001215 size=193.71MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=80.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=42B cardinality=6001215
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=80.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=42B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=80.00MB mem-reservation=8.00MB
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.71MB
+   stored statistics:
+     table: rows=6001215 size=193.71MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=80.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=42B cardinality=6001215
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=16.00MB
+Per-Host Resource Estimates: Memory=160.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=42B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Resources: mem-estimate=160.00MB mem-reservation=16.00MB
+00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
+   partitions=1/1 files=3 size=193.71MB
+   stored statistics:
+     table: rows=6001215 size=193.71MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=80.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=42B cardinality=6001215
+====
+# Small parquet files - memory reservation is reduced because of small file size.
+select string_col from functional_parquet.alltypes;
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00KB
+Per-Host Resource Estimates: Memory=16.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [functional_parquet.alltypes]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=8.00KB
+   tuple-ids=0 row-size=16B cardinality=unavailable
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=8.00KB
+Per-Host Resource Estimates: Memory=16.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=16B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB
+00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=8.00KB
+   tuple-ids=0 row-size=16B cardinality=unavailable
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=16.00KB
+Per-Host Resource Estimates: Memory=32.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=16B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB
+00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=8.00KB
+   tuple-ids=0 row-size=16B cardinality=unavailable
+====
+# Multi-column parquet scan with small files - memory reservation is reduced because of
+# small file size but a minimum amount is reserved per column.
+select int_col, float_col, string_col from functional_parquet.alltypes;
+---- PLAN
+Max Per-Host Resource Reservation: Memory=24.00KB
+Per-Host Resource Estimates: Memory=16.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=24.00KB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [functional_parquet.alltypes]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=24.00KB
+   tuple-ids=0 row-size=24B cardinality=unavailable
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=24.00KB
+Per-Host Resource Estimates: Memory=16.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=24B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=16.00MB mem-reservation=24.00KB
+00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=24.00KB
+   tuple-ids=0 row-size=24B cardinality=unavailable
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=48.00KB
+Per-Host Resource Estimates: Memory=32.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=24B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Resources: mem-estimate=32.00MB mem-reservation=48.00KB
+00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=24.00KB
+   tuple-ids=0 row-size=24B cardinality=unavailable
+====
+# Parquet scan with no materialized columns. Need reservation to scan levels to determine
+# row count.
+select 'foo' from functional_parquet.alltypes
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00KB
+Per-Host Resource Estimates: Memory=10.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=1.00MB mem-reservation=8.00KB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [functional_parquet.alltypes]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=1.00MB mem-reservation=8.00KB
+   tuple-ids=0 row-size=0B cardinality=unavailable
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=8.00KB
+Per-Host Resource Estimates: Memory=10.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=0B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=1.00MB mem-reservation=8.00KB
+00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=1.00MB mem-reservation=8.00KB
+   tuple-ids=0 row-size=0B cardinality=unavailable
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=16.00KB
+Per-Host Resource Estimates: Memory=32.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional_parquet.alltypes
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=0B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB
+00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
+   partitions=24/24 files=24 size=174.62KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/24 rows=unavailable
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=8.00KB
+   tuple-ids=0 row-size=0B cardinality=unavailable
+====
+# Parquet nested types, unnested in scan - should reserve memory for each column.
+select o_orderkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk
+from tpch_nested_parquet.customer.c_orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=48.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=48.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
+   partitions=1/1 files=4 size=292.36MB
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=48.00MB
+   tuple-ids=0 row-size=80B cardinality=1500000
+====
+# Parquet nested types, unnested in scan - don't reserve extra memory for "pos" virtual
+# column that piggy-backs on another column.
+select o_orderkey, pos
+from tpch_nested_parquet.customer.c_orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
+   partitions=1/1 files=4 size=292.36MB
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=16B cardinality=1500000
+====
+# Parquet nested types, unnested in scan - reserve memory for "pos" virtual column if it
+# is the only column materialized.
+select pos
+from tpch_nested_parquet.customer.c_orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpch_nested_parquet.customer.c_orders]
+   partitions=1/1 files=4 size=292.36MB
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=8B cardinality=1500000
+====
+# Parquet nested types, left nested in scan - should reserve memory for each scalar
+# column in the nested collection plus the top-level column.
+select c_custkey, o_orderkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority,
+    o_clerk
+from tpch_nested_parquet.customer c, c.c_orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=56.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_nested_parquet.customer
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=56.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:SUBPLAN
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1,0 row-size=104B cardinality=1500000
+|
+|--04:NESTED LOOP JOIN [CROSS JOIN]
+|  |  mem-estimate=24B mem-reservation=0B
+|  |  tuple-ids=1,0 row-size=104B cardinality=10
+|  |
+|  |--02:SINGULAR ROW SRC
+|  |     parent-subplan=01
+|  |     mem-estimate=0B mem-reservation=0B
+|  |     tuple-ids=0 row-size=24B cardinality=1
+|  |
+|  03:UNNEST [c.c_orders]
+|     parent-subplan=01
+|     mem-estimate=0B mem-reservation=0B
+|     tuple-ids=1 row-size=0B cardinality=10
+|
+00:SCAN HDFS [tpch_nested_parquet.customer c]
+   partitions=1/1 files=4 size=292.36MB
+   predicates: !empty(c.c_orders)
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns missing stats: c_orders
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=56.00MB
+   tuple-ids=0 row-size=24B cardinality=150000
+====
+# Parquet nested types, left nested in scan - should reserve memory for each scalar
+# column (excluding pos) in the nested collection plus the top-level column.
+select c_custkey, o_orderkey, pos
+from tpch_nested_parquet.customer c, c.c_orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=16.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_nested_parquet.customer
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=16.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:SUBPLAN
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1,0 row-size=40B cardinality=1500000
+|
+|--04:NESTED LOOP JOIN [CROSS JOIN]
+|  |  mem-estimate=24B mem-reservation=0B
+|  |  tuple-ids=1,0 row-size=40B cardinality=10
+|  |
+|  |--02:SINGULAR ROW SRC
+|  |     parent-subplan=01
+|  |     mem-estimate=0B mem-reservation=0B
+|  |     tuple-ids=0 row-size=24B cardinality=1
+|  |
+|  03:UNNEST [c.c_orders]
+|     parent-subplan=01
+|     mem-estimate=0B mem-reservation=0B
+|     tuple-ids=1 row-size=0B cardinality=10
+|
+00:SCAN HDFS [tpch_nested_parquet.customer c]
+   partitions=1/1 files=4 size=292.36MB
+   predicates: !empty(c.c_orders)
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns missing stats: c_orders
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=16.00MB
+   tuple-ids=0 row-size=24B cardinality=150000
+====
+# Parquet nested types, left nested in scan - should reserve memory for virtual
+# pos column in the nested collection because nothing else is materialized from
+# the collection, plus the top-level column.
+select c_custkey, pos
+from tpch_nested_parquet.customer c, c.c_orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=16.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_nested_parquet.customer
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=16.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:SUBPLAN
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1,0 row-size=32B cardinality=1500000
+|
+|--04:NESTED LOOP JOIN [CROSS JOIN]
+|  |  mem-estimate=24B mem-reservation=0B
+|  |  tuple-ids=1,0 row-size=32B cardinality=10
+|  |
+|  |--02:SINGULAR ROW SRC
+|  |     parent-subplan=01
+|  |     mem-estimate=0B mem-reservation=0B
+|  |     tuple-ids=0 row-size=24B cardinality=1
+|  |
+|  03:UNNEST [c.c_orders]
+|     parent-subplan=01
+|     mem-estimate=0B mem-reservation=0B
+|     tuple-ids=1 row-size=0B cardinality=10
+|
+00:SCAN HDFS [tpch_nested_parquet.customer c]
+   partitions=1/1 files=4 size=292.36MB
+   predicates: !empty(c.c_orders)
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns missing stats: c_orders
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=16.00MB
+   tuple-ids=0 row-size=24B cardinality=150000
+====
+# Parquet nested types, left nested in scan - should reserve memory for nested column
+# plus top-level column even though nested fields are not materialized because the
+# column levels still need to be scanned to determine the # of elements in the collection.
+select c_custkey
+from tpch_nested_parquet.customer c, c.c_orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=16.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_nested_parquet.customer
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=16.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:SUBPLAN
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1,0 row-size=24B cardinality=1500000
+|
+|--04:NESTED LOOP JOIN [CROSS JOIN]
+|  |  mem-estimate=24B mem-reservation=0B
+|  |  tuple-ids=1,0 row-size=24B cardinality=10
+|  |
+|  |--02:SINGULAR ROW SRC
+|  |     parent-subplan=01
+|  |     mem-estimate=0B mem-reservation=0B
+|  |     tuple-ids=0 row-size=24B cardinality=1
+|  |
+|  03:UNNEST [c.c_orders]
+|     parent-subplan=01
+|     mem-estimate=0B mem-reservation=0B
+|     tuple-ids=1 row-size=0B cardinality=10
+|
+00:SCAN HDFS [tpch_nested_parquet.customer c]
+   partitions=1/1 files=4 size=292.36MB
+   predicates: !empty(c.c_orders)
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns missing stats: c_orders
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=16.00MB
+   tuple-ids=0 row-size=24B cardinality=150000
+====
+# Parquet nested types, left nested in scan - should reserve memory for nested column
+# only when no top-level column is materialized.
+select o_orderkey
+from tpch_nested_parquet.customer c, c.c_orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_nested_parquet.customer
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:SUBPLAN
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=1,0 row-size=24B cardinality=1500000
+|
+|--04:NESTED LOOP JOIN [CROSS JOIN]
+|  |  mem-estimate=16B mem-reservation=0B
+|  |  tuple-ids=1,0 row-size=24B cardinality=10
+|  |
+|  |--02:SINGULAR ROW SRC
+|  |     parent-subplan=01
+|  |     mem-estimate=0B mem-reservation=0B
+|  |     tuple-ids=0 row-size=16B cardinality=1
+|  |
+|  03:UNNEST [c.c_orders]
+|     parent-subplan=01
+|     mem-estimate=0B mem-reservation=0B
+|     tuple-ids=1 row-size=0B cardinality=10
+|
+00:SCAN HDFS [tpch_nested_parquet.customer c]
+   partitions=1/1 files=4 size=292.36MB
+   predicates: !empty(c.c_orders)
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=16B cardinality=150000
+====
+# Parquet nested types with two levels of nesting materialized in scan. Should
+# reserve memory for columns at all three levels
+select c_custkey, o_orderkey, l_comment
+from tpch_nested_parquet.customer c, c.c_orders o, o.o_lineitems
+---- PLAN
+Max Per-Host Resource Reservation: Memory=24.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_nested_parquet.customer
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=24.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:SUBPLAN
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=2,1,0 row-size=64B cardinality=15000000
+|
+|--08:NESTED LOOP JOIN [CROSS JOIN]
+|  |  mem-estimate=24B mem-reservation=0B
+|  |  tuple-ids=2,1,0 row-size=64B cardinality=100
+|  |
+|  |--02:SINGULAR ROW SRC
+|  |     parent-subplan=01
+|  |     mem-estimate=0B mem-reservation=0B
+|  |     tuple-ids=0 row-size=24B cardinality=1
+|  |
+|  04:SUBPLAN
+|  |  mem-estimate=0B mem-reservation=0B
+|  |  tuple-ids=2,1 row-size=40B cardinality=100
+|  |
+|  |--07:NESTED LOOP JOIN [CROSS JOIN]
+|  |  |  mem-estimate=24B mem-reservation=0B
+|  |  |  tuple-ids=2,1 row-size=40B cardinality=10
+|  |  |
+|  |  |--05:SINGULAR ROW SRC
+|  |  |     parent-subplan=04
+|  |  |     mem-estimate=0B mem-reservation=0B
+|  |  |     tuple-ids=1 row-size=24B cardinality=1
+|  |  |
+|  |  06:UNNEST [o.o_lineitems]
+|  |     parent-subplan=04
+|  |     mem-estimate=0B mem-reservation=0B
+|  |     tuple-ids=2 row-size=0B cardinality=10
+|  |
+|  03:UNNEST [c.c_orders o]
+|     parent-subplan=01
+|     mem-estimate=0B mem-reservation=0B
+|     tuple-ids=1 row-size=0B cardinality=10
+|
+00:SCAN HDFS [tpch_nested_parquet.customer c]
+   partitions=1/1 files=4 size=292.36MB
+   predicates: !empty(c.c_orders)
+   predicates on o: !empty(o.o_lineitems)
+   stored statistics:
+     table: rows=150000 size=292.36MB
+     columns missing stats: c_orders
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=24.00MB
+   tuple-ids=0 row-size=24B cardinality=150000
+====
+# Text scan
+select * from tpch.lineitem
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpch.lineitem]
+   partitions=1/1 files=1 size=718.94MB
+   stored statistics:
+     table: rows=6001215 size=718.94MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   stored statistics:
+     table: rows=6001215 size=718.94MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=263B cardinality=6001215
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=16.00MB
+Per-Host Resource Estimates: Memory=176.00MB
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=263B cardinality=6001215
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Resources: mem-estimate=176.00MB mem-reservation=16.00MB
+00:SCAN HDFS [tpch.lineitem, RANDOM]
+   partitions=1/1 files=1 size=718.94MB
+   stored statistics:
+     table: rows=6001215 size=718.94MB
+     columns: all
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=263B cardinality=6001215
+====
+# Single column text scan - memory reservation is same as multi-column scan.
+select l_comment from tpch_parquet.lineitem
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
 Per-Host Resource Estimates: Memory=80.00MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=8.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
-   tuple-ids=0 row-size=263B cardinality=6001215
+   mem-estimate=80.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=42B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=8.00MB
 Per-Host Resource Estimates: Memory=80.00MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -28,20 +799,20 @@ PLAN-ROOT SINK
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=0B mem-reservation=0B
-|  tuple-ids=0 row-size=263B cardinality=6001215
+|  tuple-ids=0 row-size=42B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=80.00MB mem-reservation=8.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
-   tuple-ids=0 row-size=263B cardinality=6001215
+   mem-estimate=80.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=42B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=16.00MB
 Per-Host Resource Estimates: Memory=160.00MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -51,41 +822,44 @@ PLAN-ROOT SINK
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=0B mem-reservation=0B
-|  tuple-ids=0 row-size=263B cardinality=6001215
+|  tuple-ids=0 row-size=42B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=160.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=160.00MB mem-reservation=16.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
-   tuple-ids=0 row-size=263B cardinality=6001215
+   mem-estimate=80.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=42B cardinality=6001215
 ====
-# Text scan
-select * from tpch.lineitem;
+# Text scan on small files - memory reservation is reduced.
+select * from functional.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B
-Per-Host Resource Estimates: Memory=88.00MB
+Max Per-Host Resource Reservation: Memory=32.00KB
+Per-Host Resource Estimates: Memory=16.00MB
+Codegen disabled by planner
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
-00:SCAN HDFS [tpch.lineitem]
-   partitions=1/1 files=1 size=718.94MB
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
    stored statistics:
-     table: rows=6001215 size=718.94MB
+     table: rows=7300 size=478.45KB
+     partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
-   tuple-ids=0 row-size=263B cardinality=6001215
+   mem-estimate=16.00MB mem-reservation=32.00KB
+   tuple-ids=0 row-size=97B cardinality=7300
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
-Per-Host Resource Estimates: Memory=88.00MB
+Max Per-Host Resource Reservation: Memory=32.00KB
+Per-Host Resource Estimates: Memory=16.00MB
+Codegen disabled by planner
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -94,21 +868,23 @@ PLAN-ROOT SINK
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=0B mem-reservation=0B
-|  tuple-ids=0 row-size=263B cardinality=6001215
+|  tuple-ids=0 row-size=97B cardinality=7300
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
-00:SCAN HDFS [tpch.lineitem, RANDOM]
-   partitions=1/1 files=1 size=718.94MB
+Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB
+00:SCAN HDFS [functional.alltypes, RANDOM]
+   partitions=24/24 files=24 size=478.45KB
    stored statistics:
-     table: rows=6001215 size=718.94MB
+     table: rows=7300 size=478.45KB
+     partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
-   tuple-ids=0 row-size=263B cardinality=6001215
+   mem-estimate=16.00MB mem-reservation=32.00KB
+   tuple-ids=0 row-size=97B cardinality=7300
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B
-Per-Host Resource Estimates: Memory=176.00MB
+Max Per-Host Resource Reservation: Memory=64.00KB
+Per-Host Resource Estimates: Memory=32.00MB
+Codegen disabled by planner
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -117,18 +893,310 @@ PLAN-ROOT SINK
 |
 01:EXCHANGE [UNPARTITIONED]
 |  mem-estimate=0B mem-reservation=0B
-|  tuple-ids=0 row-size=263B cardinality=6001215
+|  tuple-ids=0 row-size=97B cardinality=7300
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
-00:SCAN HDFS [tpch.lineitem, RANDOM]
-   partitions=1/1 files=1 size=718.94MB
+Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB
+00:SCAN HDFS [functional.alltypes, RANDOM]
+   partitions=24/24 files=24 size=478.45KB
    stored statistics:
-     table: rows=6001215 size=718.94MB
+     table: rows=7300 size=478.45KB
+     partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
-   tuple-ids=0 row-size=263B cardinality=6001215
+   mem-estimate=16.00MB mem-reservation=32.00KB
+   tuple-ids=0 row-size=97B cardinality=7300
+====
+# Avro scan.
+select * from tpch_avro.orders
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_avro.orders
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpch_avro.orders]
+   partitions=1/1 files=2 size=156.92MB
+   stored statistics:
+     table: rows=unavailable size=156.92MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=108B cardinality=unavailable
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=88.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_avro.orders
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=108B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
+Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
+00:SCAN HDFS [tpch_avro.orders, RANDOM]
+   partitions=1/1 files=2 size=156.92MB
+   stored statistics:
+     table: rows=unavailable size=156.92MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=108B cardinality=unavailable
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=16.00MB
+Per-Host Resource Estimates: Memory=176.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_avro.orders
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=108B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
+Per-Host Resources: mem-estimate=176.00MB mem-reservation=16.00MB
+00:SCAN HDFS [tpch_avro.orders, RANDOM]
+   partitions=1/1 files=2 size=156.92MB
+   stored statistics:
+     table: rows=unavailable size=156.92MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=88.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=108B cardinality=unavailable
+====
+# RC scan.
+select * from tpch_rc.customer
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=32.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_rc.customer
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpch_rc.customer]
+   partitions=1/1 files=1 size=22.48MB
+   stored statistics:
+     table: rows=unavailable size=22.48MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=32.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=98B cardinality=unavailable
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=32.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_rc.customer
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=98B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00MB
+00:SCAN HDFS [tpch_rc.customer, RANDOM]
+   partitions=1/1 files=1 size=22.48MB
+   stored statistics:
+     table: rows=unavailable size=22.48MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=32.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=98B cardinality=unavailable
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=16.00MB
+Per-Host Resource Estimates: Memory=64.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpch_rc.customer
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=98B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+Per-Host Resources: mem-estimate=64.00MB mem-reservation=16.00MB
+00:SCAN HDFS [tpch_rc.customer, RANDOM]
+   partitions=1/1 files=1 size=22.48MB
+   stored statistics:
+     table: rows=unavailable size=22.48MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=32.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=98B cardinality=unavailable
+====
+# Seq scan.
+select * from tpcds_seq_snap.web_returns
+---- PLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=16.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpcds_seq_snap.web_returns
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00MB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [tpcds_seq_snap.web_returns]
+   partitions=1/1 files=1 size=6.55MB
+   stored statistics:
+     table: rows=unavailable size=6.55MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=104B cardinality=unavailable
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=16.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpcds_seq_snap.web_returns
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=104B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
+Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00MB
+00:SCAN HDFS [tpcds_seq_snap.web_returns, RANDOM]
+   partitions=1/1 files=1 size=6.55MB
+   stored statistics:
+     table: rows=unavailable size=6.55MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=104B cardinality=unavailable
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=16.00MB
+Per-Host Resource Estimates: Memory=32.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+tpcds_seq_snap.web_returns
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=104B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
+Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00MB
+00:SCAN HDFS [tpcds_seq_snap.web_returns, RANDOM]
+   partitions=1/1 files=1 size=6.55MB
+   stored statistics:
+     table: rows=unavailable size=6.55MB
+     columns: unavailable
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=8.00MB
+   tuple-ids=0 row-size=104B cardinality=unavailable
+====
+# Mixed table format scan
+select * from functional.alltypesmixedformat
+---- PLAN
+Max Per-Host Resource Reservation: Memory=32.00KB
+Per-Host Resource Estimates: Memory=16.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional.alltypesmixedformat
+
+F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+00:SCAN HDFS [functional.alltypesmixedformat]
+   partitions=3/3 files=3 size=58.36KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/3 rows=unavailable
+     columns missing stats: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=32.00KB
+   tuple-ids=0 row-size=88B cardinality=unavailable
+---- DISTRIBUTEDPLAN
+Max Per-Host Resource Reservation: Memory=32.00KB
+Per-Host Resource Estimates: Memory=16.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional.alltypesmixedformat
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=88B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
+Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB
+00:SCAN HDFS [functional.alltypesmixedformat, RANDOM]
+   partitions=3/3 files=3 size=58.36KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/3 rows=unavailable
+     columns missing stats: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=32.00KB
+   tuple-ids=0 row-size=88B cardinality=unavailable
+---- PARALLELPLANS
+Max Per-Host Resource Reservation: Memory=64.00KB
+Per-Host Resource Estimates: Memory=32.00MB
+WARNING: The following tables are missing relevant table and/or column statistics.
+functional.alltypesmixedformat
+
+F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
+|  Per-Host Resources: mem-estimate=0B mem-reservation=0B
+PLAN-ROOT SINK
+|  mem-estimate=0B mem-reservation=0B
+|
+01:EXCHANGE [UNPARTITIONED]
+|  mem-estimate=0B mem-reservation=0B
+|  tuple-ids=0 row-size=88B cardinality=unavailable
+|
+F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
+Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB
+00:SCAN HDFS [functional.alltypesmixedformat, RANDOM]
+   partitions=3/3 files=3 size=58.36KB
+   stored statistics:
+     table: rows=unavailable size=unavailable
+     partitions: 0/3 rows=unavailable
+     columns missing stats: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
+   extrapolated-rows=disabled
+   mem-estimate=16.00MB mem-reservation=32.00KB
+   tuple-ids=0 row-size=88B cardinality=unavailable
 ====
 # HBase scan
 select * from functional_hbase.alltypes
@@ -258,11 +1326,11 @@ select * from tpch.lineitem
 union all
 select * from tpch.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=8.00MB
 Per-Host Resource Estimates: Memory=88.00MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -277,7 +1345,7 @@ PLAN-ROOT SINK
 |       table: rows=6001215 size=718.94MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=263B cardinality=6001215
 |
 01:SCAN HDFS [tpch.lineitem]
@@ -286,10 +1354,10 @@ PLAN-ROOT SINK
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=8.00MB
 Per-Host Resource Estimates: Memory=88.00MB
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -302,7 +1370,7 @@ PLAN-ROOT SINK
 |  tuple-ids=2 row-size=263B cardinality=12002430
 |
 F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
 00:UNION
 |  pass-through-operands: all
 |  mem-estimate=0B mem-reservation=0B
@@ -314,7 +1382,7 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
 |       table: rows=6001215 size=718.94MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=263B cardinality=6001215
 |
 01:SCAN HDFS [tpch.lineitem, RANDOM]
@@ -323,10 +1391,10 @@ Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=16.00MB
 Per-Host Resource Estimates: Memory=176.00MB
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -339,7 +1407,7 @@ PLAN-ROOT SINK
 |  tuple-ids=2 row-size=263B cardinality=12002430
 |
 F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=176.00MB mem-reservation=16.00MB
 00:UNION
 |  pass-through-operands: all
 |  mem-estimate=0B mem-reservation=0B
@@ -351,7 +1419,7 @@ Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
 |       table: rows=6001215 size=718.94MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=263B cardinality=6001215
 |
 01:SCAN HDFS [tpch.lineitem, RANDOM]
@@ -360,7 +1428,7 @@ Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Grouping aggregation
@@ -368,11 +1436,11 @@ select l_orderkey, count(*)
 from tpch_parquet.lineitem
 group by l_orderkey
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=34.00MB
+Max Per-Host Resource Reservation: Memory=42.00MB
 Per-Host Resource Estimates: Memory=114.00MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=114.00MB mem-reservation=34.00MB
+|  Per-Host Resources: mem-estimate=114.00MB mem-reservation=42.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -383,15 +1451,15 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=16B cardinality=1563438
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=42.50MB
+Max Per-Host Resource Reservation: Memory=50.50MB
 Per-Host Resource Estimates: Memory=124.00MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -416,7 +1484,7 @@ Per-Host Resources: mem-estimate=10.00MB mem-reservation=8.50MB
 |  tuple-ids=1 row-size=16B cardinality=1563438
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=114.00MB mem-reservation=34.00MB
+Per-Host Resources: mem-estimate=114.00MB mem-reservation=42.00MB
 01:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: l_orderkey
@@ -424,15 +1492,15 @@ Per-Host Resources: mem-estimate=114.00MB mem-reservation=34.00MB
 |  tuple-ids=1 row-size=16B cardinality=1563438
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=77.50MB
+Max Per-Host Resource Reservation: Memory=93.50MB
 Per-Host Resource Estimates: Memory=248.00MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -457,7 +1525,7 @@ Per-Host Resources: mem-estimate=20.00MB mem-reservation=9.50MB
 |  tuple-ids=1 row-size=16B cardinality=1563438
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=228.00MB mem-reservation=68.00MB
+Per-Host Resources: mem-estimate=228.00MB mem-reservation=84.00MB
 01:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: l_orderkey
@@ -465,22 +1533,22 @@ Per-Host Resources: mem-estimate=228.00MB mem-reservation=68.00MB
 |  tuple-ids=1 row-size=16B cardinality=1563438
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ====
 # Non-grouping aggregation with zero-slot parquet scan
 select count(*) from tpch_parquet.lineitem
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B
-Per-Host Resource Estimates: Memory=11.00MB
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=18.00MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=11.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=18.00MB mem-reservation=8.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -490,16 +1558,16 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=1.00MB mem-reservation=0B
+   mem-estimate=8.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
-Per-Host Resource Estimates: Memory=21.00MB
+Max Per-Host Resource Reservation: Memory=8.00MB
+Per-Host Resource Estimates: Memory=28.00MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=10.00MB mem-reservation=0B
@@ -516,22 +1584,22 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=11.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=18.00MB mem-reservation=8.00MB
 01:AGGREGATE
 |  output: sum_init_zero(tpch_parquet.lineitem.parquet-stats: num_rows)
 |  mem-estimate=10.00MB mem-reservation=0B spill-buffer=2.00MB
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=1.00MB mem-reservation=0B
+   mem-estimate=8.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=16.00MB
 Per-Host Resource Estimates: Memory=190.00MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -549,19 +1617,19 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=180.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=180.00MB mem-reservation=16.00MB
 01:AGGREGATE
 |  output: sum_init_zero(tpch_parquet.lineitem.parquet-stats: num_rows)
 |  mem-estimate=10.00MB mem-reservation=0B spill-buffer=2.00MB
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ====
 # Sort
@@ -569,11 +1637,11 @@ select *
 from tpch_parquet.lineitem
 order by l_comment
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=12.00MB
+Max Per-Host Resource Reservation: Memory=84.00MB
 Per-Host Resource Estimates: Memory=120.00MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=120.00MB mem-reservation=12.00MB
+|  Per-Host Resources: mem-estimate=120.00MB mem-reservation=84.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -583,15 +1651,15 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=12.00MB
+Max Per-Host Resource Reservation: Memory=84.00MB
 Per-Host Resource Estimates: Memory=120.00MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -605,22 +1673,22 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=120.00MB mem-reservation=12.00MB
+Per-Host Resources: mem-estimate=120.00MB mem-reservation=84.00MB
 01:SORT
 |  order by: l_comment ASC
 |  mem-estimate=40.00MB mem-reservation=12.00MB spill-buffer=2.00MB
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=24.00MB
+Max Per-Host Resource Reservation: Memory=168.00MB
 Per-Host Resource Estimates: Memory=240.00MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -634,19 +1702,19 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=240.00MB mem-reservation=24.00MB
+Per-Host Resources: mem-estimate=240.00MB mem-reservation=168.00MB
 01:SORT
 |  order by: l_comment ASC
 |  mem-estimate=40.00MB mem-reservation=12.00MB spill-buffer=2.00MB
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # TOP-N
@@ -655,11 +1723,11 @@ from tpch_parquet.lineitem
 order by l_comment
 limit 100
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=72.00MB
 Per-Host Resource Estimates: Memory=80.03MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=80.03MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=80.03MB mem-reservation=72.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -669,15 +1737,15 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=263B cardinality=100
 |
 00:SCAN HDFS [tpch_parquet.lineitem]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=72.00MB
 Per-Host Resource Estimates: Memory=80.03MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -692,22 +1760,22 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=263B cardinality=100
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=80.03MB mem-reservation=0B
+Per-Host Resources: mem-estimate=80.03MB mem-reservation=72.00MB
 01:TOP-N [LIMIT=100]
 |  order by: l_comment ASC
 |  mem-estimate=25.66KB mem-reservation=0B
 |  tuple-ids=1 row-size=263B cardinality=100
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=144.00MB
 Per-Host Resource Estimates: Memory=160.05MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -722,30 +1790,30 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=263B cardinality=100
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=160.05MB mem-reservation=0B
+Per-Host Resources: mem-estimate=160.05MB mem-reservation=144.00MB
 01:TOP-N [LIMIT=100]
 |  order by: l_comment ASC
 |  mem-estimate=25.66KB mem-reservation=0B
 |  tuple-ids=1 row-size=263B cardinality=100
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Broadcast Hash Join
 select *
 from tpch.lineitem inner join tpch.orders on l_orderkey = o_orderkey
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=35.00MB
+Max Per-Host Resource Reservation: Memory=51.00MB
 Per-Host Resource Estimates: Memory=477.41MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=477.41MB mem-reservation=35.00MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=477.41MB mem-reservation=51.00MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -762,7 +1830,7 @@ PLAN-ROOT SINK
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch.lineitem]
@@ -772,10 +1840,10 @@ PLAN-ROOT SINK
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=35.00MB
+Max Per-Host Resource Reservation: Memory=51.00MB
 Per-Host Resource Estimates: Memory=477.41MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -788,7 +1856,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=454B cardinality=5757710
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=389.41MB mem-reservation=35.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=389.41MB mem-reservation=43.00MB runtime-filters-memory=1.00MB
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
@@ -801,14 +1869,14 @@ Per-Host Resources: mem-estimate=389.41MB mem-reservation=35.00MB runtime-filter
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     partitions=1/1 files=1 size=162.56MB
 |     stored statistics:
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch.lineitem, RANDOM]
@@ -818,10 +1886,10 @@ Per-Host Resources: mem-estimate=389.41MB mem-reservation=35.00MB runtime-filter
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=70.00MB
+Max Per-Host Resource Reservation: Memory=102.00MB
 Per-Host Resource Estimates: Memory=954.83MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -834,7 +1902,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=454B cardinality=5757710
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=778.83MB mem-reservation=70.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=778.83MB mem-reservation=86.00MB runtime-filters-memory=1.00MB
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
@@ -855,14 +1923,14 @@ Per-Host Resources: mem-estimate=778.83MB mem-reservation=70.00MB runtime-filter
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=16.00MB
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     partitions=1/1 files=1 size=162.56MB
 |     stored statistics:
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch.lineitem, RANDOM]
@@ -872,18 +1940,18 @@ Per-Host Resources: mem-estimate=778.83MB mem-reservation=70.00MB runtime-filter
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Shuffle Hash Join
 select *
 from tpch.lineitem inner join /* +shuffle */ tpch.orders on l_orderkey = o_orderkey
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=35.00MB
+Max Per-Host Resource Reservation: Memory=51.00MB
 Per-Host Resource Estimates: Memory=477.41MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=477.41MB mem-reservation=35.00MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=477.41MB mem-reservation=51.00MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -900,7 +1968,7 @@ PLAN-ROOT SINK
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch.lineitem]
@@ -910,10 +1978,10 @@ PLAN-ROOT SINK
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=36.00MB
+Max Per-Host Resource Reservation: Memory=52.00MB
 Per-Host Resource Estimates: Memory=278.14MB
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -939,14 +2007,14 @@ Per-Host Resources: mem-estimate=101.14MB mem-reservation=35.00MB runtime-filter
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     partitions=1/1 files=1 size=162.56MB
 |     stored statistics:
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 03:EXCHANGE [HASH(l_orderkey)]
@@ -954,7 +2022,7 @@ Per-Host Resources: mem-estimate=101.14MB mem-reservation=35.00MB runtime-filter
 |  tuple-ids=0 row-size=263B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=89.00MB mem-reservation=1.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=89.00MB mem-reservation=9.00MB runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch.lineitem, RANDOM]
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000[bloom] -> l_orderkey
@@ -962,10 +2030,10 @@ Per-Host Resources: mem-estimate=89.00MB mem-reservation=1.00MB runtime-filters-
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=72.00MB
+Max Per-Host Resource Reservation: Memory=104.00MB
 Per-Host Resource Estimates: Memory=456.14MB
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -999,14 +2067,14 @@ Per-Host Resources: mem-estimate=102.14MB mem-reservation=70.00MB runtime-filter
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=16.00MB
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     partitions=1/1 files=1 size=162.56MB
 |     stored statistics:
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 03:EXCHANGE [HASH(l_orderkey)]
@@ -1014,7 +2082,7 @@ Per-Host Resources: mem-estimate=102.14MB mem-reservation=70.00MB runtime-filter
 |  tuple-ids=0 row-size=263B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=178.00MB mem-reservation=2.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=178.00MB mem-reservation=18.00MB runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch.lineitem, RANDOM]
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000[bloom] -> l_orderkey
@@ -1022,18 +2090,18 @@ Per-Host Resources: mem-estimate=178.00MB mem-reservation=2.00MB runtime-filters
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Nested loop join
 select *
 from tpch.lineitem, tpch.orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=16.00MB
 Per-Host Resource Estimates: Memory=449.10MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=449.10MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=449.10MB mem-reservation=16.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -1047,7 +2115,7 @@ PLAN-ROOT SINK
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch.lineitem]
@@ -1056,10 +2124,10 @@ PLAN-ROOT SINK
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=16.00MB
 Per-Host Resource Estimates: Memory=449.10MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -1072,7 +2140,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=454B cardinality=9001822500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=361.10MB mem-reservation=0B
+Per-Host Resources: mem-estimate=361.10MB mem-reservation=8.00MB
 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  mem-estimate=273.10MB mem-reservation=0B
 |  tuple-ids=0,1 row-size=454B cardinality=9001822500000
@@ -1082,14 +2150,14 @@ Per-Host Resources: mem-estimate=361.10MB mem-reservation=0B
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=8.00MB
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     partitions=1/1 files=1 size=162.56MB
 |     stored statistics:
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch.lineitem, RANDOM]
@@ -1098,10 +2166,10 @@ Per-Host Resources: mem-estimate=361.10MB mem-reservation=0B
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=32.00MB
 Per-Host Resource Estimates: Memory=898.21MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -1114,7 +2182,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=454B cardinality=9001822500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=722.21MB mem-reservation=0B
+Per-Host Resources: mem-estimate=722.21MB mem-reservation=16.00MB
 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  join table id: 00
 |  mem-estimate=273.10MB mem-reservation=0B
@@ -1131,14 +2199,14 @@ Per-Host Resources: mem-estimate=722.21MB mem-reservation=0B
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=16.00MB
 |  01:SCAN HDFS [tpch.orders, RANDOM]
 |     partitions=1/1 files=1 size=162.56MB
 |     stored statistics:
 |       table: rows=1500000 size=162.56MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=88.00MB mem-reservation=0B
+|     mem-estimate=88.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch.lineitem, RANDOM]
@@ -1147,7 +2215,7 @@ Per-Host Resources: mem-estimate=722.21MB mem-reservation=0B
      table: rows=6001215 size=718.94MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Empty set node
@@ -1196,12 +2264,12 @@ PLAN-ROOT SINK
 select max(tinyint_col) over(partition by int_col)
 from functional.alltypes
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=10.00MB
+Max Per-Host Resource Reservation: Memory=10.03MB
 Per-Host Resource Estimates: Memory=26.00MB
 Codegen disabled by planner
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=10.00MB
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=10.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -1223,10 +2291,10 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=5B cardinality=7300
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=10.00MB
+Max Per-Host Resource Reservation: Memory=10.03MB
 Per-Host Resource Estimates: Memory=26.00MB
 Codegen disabled by planner
 
@@ -1257,7 +2325,7 @@ Per-Host Resources: mem-estimate=10.00MB mem-reservation=10.00MB
 |  tuple-ids=0 row-size=5B cardinality=7300
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB
 00:SCAN HDFS [functional.alltypes, RANDOM]
    partitions=24/24 files=24 size=478.45KB
    stored statistics:
@@ -1265,10 +2333,10 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=5B cardinality=7300
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=20.00MB
+Max Per-Host Resource Reservation: Memory=20.06MB
 Per-Host Resource Estimates: Memory=52.00MB
 Codegen disabled by planner
 
@@ -1299,7 +2367,7 @@ Per-Host Resources: mem-estimate=20.00MB mem-reservation=20.00MB
 |  tuple-ids=0 row-size=5B cardinality=7300
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB
 00:SCAN HDFS [functional.alltypes, RANDOM]
    partitions=24/24 files=24 size=478.45KB
    stored statistics:
@@ -1307,7 +2375,7 @@ Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=5B cardinality=7300
 ====
 # Pipeline of blocking operators from analytic fns. Blocking operators break
@@ -1317,11 +2385,11 @@ select *, row_number() over (order by o_totalprice) rnum_price,
   row_number() over (order by o_orderpriority) rnum_priority
 from tpch_parquet.orders
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=36.00MB
-Per-Host Resource Estimates: Memory=70.00MB
+Max Per-Host Resource Reservation: Memory=72.00MB
+Per-Host Resource Estimates: Memory=78.00MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=70.00MB mem-reservation=36.00MB
+|  Per-Host Resources: mem-estimate=78.00MB mem-reservation=72.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -1367,11 +2435,11 @@ PLAN-ROOT SINK
      table: rows=1500000 size=54.07MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=40.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=48.00MB
    tuple-ids=0 row-size=191B cardinality=1500000
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=44.00MB
-Per-Host Resource Estimates: Memory=102.00MB
+Max Per-Host Resource Reservation: Memory=92.00MB
+Per-Host Resource Estimates: Memory=110.00MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=44.00MB mem-reservation=32.00MB
@@ -1415,7 +2483,7 @@ PLAN-ROOT SINK
 |  tuple-ids=6 row-size=191B cardinality=1500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-Per-Host Resources: mem-estimate=58.00MB mem-reservation=12.00MB
+Per-Host Resources: mem-estimate=66.00MB mem-reservation=60.00MB
 01:SORT
 |  order by: o_totalprice ASC
 |  mem-estimate=18.00MB mem-reservation=12.00MB spill-buffer=2.00MB
@@ -1427,11 +2495,11 @@ Per-Host Resources: mem-estimate=58.00MB mem-reservation=12.00MB
      table: rows=1500000 size=54.07MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=40.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=48.00MB
    tuple-ids=0 row-size=191B cardinality=1500000
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=56.00MB
-Per-Host Resource Estimates: Memory=160.00MB
+Max Per-Host Resource Reservation: Memory=152.00MB
+Per-Host Resource Estimates: Memory=176.00MB
 
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=44.00MB mem-reservation=32.00MB
@@ -1475,7 +2543,7 @@ PLAN-ROOT SINK
 |  tuple-ids=6 row-size=191B cardinality=1500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-Per-Host Resources: mem-estimate=116.00MB mem-reservation=24.00MB
+Per-Host Resources: mem-estimate=132.00MB mem-reservation=120.00MB
 01:SORT
 |  order by: o_totalprice ASC
 |  mem-estimate=18.00MB mem-reservation=12.00MB spill-buffer=2.00MB
@@ -1487,7 +2555,7 @@ Per-Host Resources: mem-estimate=116.00MB mem-reservation=24.00MB
      table: rows=1500000 size=54.07MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=40.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=48.00MB
    tuple-ids=0 row-size=191B cardinality=1500000
 ====
 # Union with non-trivial branches: each branch executes sequentially within fragment.
@@ -1503,11 +2571,11 @@ select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
 from tpch_parquet.lineitem join tpch_parquet.orders on l_orderkey = o_orderkey
 where l_shipmode = 'F'
 ---- PLAN
-Max Per-Host Resource Reservation: Memory=54.00MB
+Max Per-Host Resource Reservation: Memory=102.00MB
 Per-Host Resource Estimates: Memory=142.58MB
 
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=142.58MB mem-reservation=54.00MB runtime-filters-memory=3.00MB
+|  Per-Host Resources: mem-estimate=142.58MB mem-reservation=102.00MB runtime-filters-memory=3.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -1529,20 +2597,20 @@ PLAN-ROOT SINK
 |  |       table: rows=1500000 size=54.07MB
 |  |       columns: all
 |  |     extrapolated-rows=disabled
-|  |     mem-estimate=40.00MB mem-reservation=0B
+|  |     mem-estimate=40.00MB mem-reservation=8.00MB
 |  |     tuple-ids=6 row-size=8B cardinality=1500000
 |  |
 |  08:SCAN HDFS [tpch_parquet.lineitem]
-|     partitions=1/1 files=3 size=193.73MB
+|     partitions=1/1 files=3 size=193.71MB
 |     predicates: l_shipmode = 'F'
 |     runtime filters: RF004[bloom] -> l_orderkey
 |     stored statistics:
-|       table: rows=6001215 size=193.73MB
+|       table: rows=6001215 size=193.71MB
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet statistics predicates: l_shipmode = 'F'
 |     parquet dictionary predicates: l_shipmode = 'F'
-|     mem-estimate=80.00MB mem-reservation=0B
+|     mem-estimate=80.00MB mem-reservation=48.00MB
 |     tuple-ids=5 row-size=91B cardinality=857316
 |
 |--07:HASH JOIN [INNER JOIN]
@@ -1561,17 +2629,17 @@ PLAN-ROOT SINK
 |  |     extrapolated-rows=disabled
 |  |     parquet statistics predicates: o_orderpriority = '2-HIGH'
 |  |     parquet dictionary predicates: o_orderpriority = '2-HIGH'
-|  |     mem-estimate=40.00MB mem-reservation=0B
+|  |     mem-estimate=40.00MB mem-reservation=16.00MB
 |  |     tuple-ids=4 row-size=32B cardinality=300000
 |  |
 |  05:SCAN HDFS [tpch_parquet.lineitem]
-|     partitions=1/1 files=3 size=193.73MB
+|     partitions=1/1 files=3 size=193.71MB
 |     runtime filters: RF002[bloom] -> l_orderkey
 |     stored statistics:
-|       table: rows=6001215 size=193.73MB
+|       table: rows=6001215 size=193.71MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=80.00MB mem-reservation=0B
+|     mem-estimate=80.00MB mem-reservation=40.00MB
 |     tuple-ids=3 row-size=70B cardinality=6001215
 |
 04:AGGREGATE [FINALIZE]
@@ -1592,23 +2660,23 @@ PLAN-ROOT SINK
 |       table: rows=1500000 size=54.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=40.00MB mem-reservation=0B
+|     mem-estimate=40.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=8B cardinality=1500000
 |
 01:SCAN HDFS [tpch_parquet.lineitem]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.71MB
    predicates: l_tax > 10
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.71MB
      columns: all
    extrapolated-rows=disabled
    parquet statistics predicates: l_tax > 10
    parquet dictionary predicates: l_tax > 10
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=48.00MB
    tuple-ids=0 row-size=78B cardinality=600122
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=76.75MB
+Max Per-Host Resource Reservation: Memory=187.75MB
 Per-Host Resource Estimates: Memory=348.33MB
 
 F09:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -1621,7 +2689,7 @@ PLAN-ROOT SINK
 |  tuple-ids=7 row-size=70B cardinality=2549844
 |
 F08:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=99.00MB mem-reservation=36.00MB runtime-filters-memory=2.00MB
+Per-Host Resources: mem-estimate=99.00MB mem-reservation=67.00MB runtime-filters-memory=2.00MB
 00:UNION
 |  pass-through-operands: 14
 |  mem-estimate=0B mem-reservation=0B
@@ -1639,27 +2707,27 @@ Per-Host Resources: mem-estimate=99.00MB mem-reservation=36.00MB runtime-filters
 |  |  |  tuple-ids=6 row-size=8B cardinality=1500000
 |  |  |
 |  |  F07:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B
+|  |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB
 |  |  09:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |  |     partitions=1/1 files=2 size=54.07MB
 |  |     stored statistics:
 |  |       table: rows=1500000 size=54.07MB
 |  |       columns: all
 |  |     extrapolated-rows=disabled
-|  |     mem-estimate=40.00MB mem-reservation=0B
+|  |     mem-estimate=40.00MB mem-reservation=8.00MB
 |  |     tuple-ids=6 row-size=8B cardinality=1500000
 |  |
 |  08:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-|     partitions=1/1 files=3 size=193.73MB
+|     partitions=1/1 files=3 size=193.71MB
 |     predicates: l_shipmode = 'F'
 |     runtime filters: RF004[bloom] -> l_orderkey
 |     stored statistics:
-|       table: rows=6001215 size=193.73MB
+|       table: rows=6001215 size=193.71MB
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet statistics predicates: l_shipmode = 'F'
 |     parquet dictionary predicates: l_shipmode = 'F'
-|     mem-estimate=80.00MB mem-reservation=0B
+|     mem-estimate=80.00MB mem-reservation=48.00MB
 |     tuple-ids=5 row-size=91B cardinality=857316
 |
 |--07:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1674,7 +2742,7 @@ Per-Host Resources: mem-estimate=99.00MB mem-reservation=36.00MB runtime-filters
 |  |  |  tuple-ids=4 row-size=32B cardinality=300000
 |  |  |
 |  |  F05:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B
+|  |  Per-Host Resources: mem-estimate=40.00MB mem-reservation=16.00MB
 |  |  06:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |  |     partitions=1/1 files=2 size=54.07MB
 |  |     predicates: o_orderpriority = '2-HIGH'
@@ -1684,17 +2752,17 @@ Per-Host Resources: mem-estimate=99.00MB mem-reservation=36.00MB runtime-filters
 |  |     extrapolated-rows=disabled
 |  |     parquet statistics predicates: o_orderpriority = '2-HI

<TRUNCATED>

[09/15] impala git commit: IMPALA-4167: Support insert plan hints for CREATE TABLE AS SELECT

Posted by ta...@apache.org.
IMPALA-4167: Support insert plan hints for CREATE TABLE AS SELECT

This change adds support for "clustered", "noclustered", "shuffle" and
"noshuffle" hints in CTAS statement.

Example:
create /*+ clustered,noshuffle */ table t partitioned by (year, month)
as select * from functional.alltypes

The effect of these hints are the same as in insert statements:

clustered:
Sort locally by partition expression before insert to ensure that only
one partition is written at a time. The goal is to reduce the number of
files kept open / buffers kept in memory simultaneously.

noclustered:
Do not sort by primary key before insert to Kudu table. No effect on HDFS
tables currently, as this is their default behavior.

shuffle:
Forces the planner to add an exchange node that repartitions by the
partition expression of the output table. This means that a partition
will be written only by a single node, which minimizes the global
number of simultaneous writes.
If only one partition is written (because all partitioning columns
are constant or the target table is not partitioned), then the shuffle
hint leads to a plan where all rows are merged at the coordinator where
the table sink is executed.

noshuffle:
Do not add exchange node before insert to partitioned tables.

The parser needed some modifications to be able to differentiate between
CREATE statements that allow hints (CTAS), and CREATE statements that
do not (every other type of CREATE statements). As a result, KW_CREATE
was moved from tbl_def_without_col_defs to statement rules.

Testing:

The parser tests mirror the tests of INSERT, while analysis and planner
tests are minimal, as the underlying logic is the same as for INSERT.

Query tests are not created, as the hints have no effect on
the DDL part of CTAS, and the actual query ran is the same as in
the insert case.

Change-Id: I8d74bca999da8ae1bb89427c70841f33e3c56ab0
Reviewed-on: http://gerrit.cloudera.org:8080/8400
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins
Reviewed-on: http://gerrit.cloudera.org:8080/9422
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Tim Armstrong <ta...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/07d9f71a
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/07d9f71a
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/07d9f71a

Branch: refs/heads/2.x
Commit: 07d9f71a9359d0b4b668088678bbbfb59dff9142
Parents: 31d197a
Author: Csaba Ringhofer <cs...@cloudera.com>
Authored: Thu Oct 26 22:15:51 2017 +0200
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Fri Feb 23 22:51:01 2018 +0000

----------------------------------------------------------------------
 fe/src/main/cup/sql-parser.cup                  | 54 ++++++++----
 .../analysis/CreateTableAsSelectStmt.java       | 37 ++++++---
 .../apache/impala/analysis/AnalyzeDDLTest.java  | 28 +++++++
 .../impala/analysis/AnalyzeStmtsTest.java       |  7 +-
 .../org/apache/impala/analysis/ParserTest.java  | 86 +++++++++++++-------
 .../org/apache/impala/analysis/ToSqlTest.java   |  7 +-
 .../apache/impala/common/FrontendTestBase.java  |  5 ++
 .../queries/PlannerTest/ddl.test                | 31 +++++++
 .../queries/PlannerTest/kudu.test               | 21 +++++
 9 files changed, 211 insertions(+), 65 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/fe/src/main/cup/sql-parser.cup
----------------------------------------------------------------------
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index 38183f8..9f55dda 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -383,7 +383,7 @@ nonterminal Subquery subquery;
 nonterminal JoinOperator join_operator;
 nonterminal opt_inner, opt_outer;
 nonterminal PlanHint plan_hint;
-nonterminal List<PlanHint> opt_plan_hints, plan_hint_list;
+nonterminal List<PlanHint> plan_hints, opt_plan_hints, plan_hint_list;
 nonterminal TypeDef type_def;
 nonterminal Type type;
 nonterminal Expr sign_chain_expr;
@@ -417,6 +417,7 @@ nonterminal DropStatsStmt drop_stats_stmt;
 nonterminal DropTableOrViewStmt drop_tbl_or_view_stmt;
 nonterminal CreateDbStmt create_db_stmt;
 nonterminal CreateTableAsSelectStmt create_tbl_as_select_stmt;
+nonterminal CreateTableAsSelectStmt.CtasParams create_tbl_as_select_params;
 nonterminal CreateTableLikeStmt create_tbl_like_stmt;
 nonterminal CreateTableStmt create_tbl_stmt;
 nonterminal TableDef tbl_def_without_col_defs, tbl_def_with_col_defs;
@@ -1103,13 +1104,30 @@ create_db_stmt ::=
   {: RESULT = new CreateDbStmt(db_name, comment, location, if_not_exists); :}
   ;
 
+
+// Merging the two cases using opt_plan_hints would lead to reduce-reduce conflict,
+// because if there are no hints, CTAS cannot be distinguished from normal CREATE
+// statements until the AS SELECT part, but the decision whether to reduce empty string
+// to opt_plan_hints must happen before reaching that part.
 create_tbl_as_select_stmt ::=
+  KW_CREATE create_tbl_as_select_params:ctas_params
+  {:
+    RESULT = new CreateTableAsSelectStmt(ctas_params, null);
+  :}
+  | KW_CREATE plan_hints:hints create_tbl_as_select_params:ctas_params
+  {:
+    RESULT = new CreateTableAsSelectStmt(ctas_params, hints);
+  :}
+  ;
+
+create_tbl_as_select_params ::=
   tbl_def_without_col_defs:tbl_def
   tbl_options:options
   KW_AS query_stmt:select_stmt
   {:
     tbl_def.setOptions(options);
-    RESULT = new CreateTableAsSelectStmt(new CreateTableStmt(tbl_def), select_stmt, null);
+    RESULT = new CreateTableAsSelectStmt.CtasParams(new CreateTableStmt(tbl_def),
+        select_stmt, null);
   :}
   | tbl_def_without_col_defs:tbl_def
     // An optional clause cannot be used directly below because it would conflict with
@@ -1122,7 +1140,8 @@ create_tbl_as_select_stmt ::=
     tbl_def.getPrimaryKeyColumnNames().addAll(primary_keys);
     tbl_def.getKuduPartitionParams().addAll(partition_params.getKuduPartitionParams());
     tbl_def.setOptions(options);
-    RESULT = new CreateTableAsSelectStmt(new CreateTableStmt(tbl_def), select_stmt, null);
+    RESULT = new CreateTableAsSelectStmt.CtasParams(new CreateTableStmt(tbl_def),
+        select_stmt, null);
   :}
   | tbl_def_without_col_defs:tbl_def
     KW_PARTITIONED KW_BY LPAREN ident_list:partition_cols RPAREN
@@ -1130,19 +1149,19 @@ create_tbl_as_select_stmt ::=
     KW_AS query_stmt:select_stmt
   {:
     tbl_def.setOptions(options);
-    RESULT = new CreateTableAsSelectStmt(new CreateTableStmt(tbl_def),
+    RESULT = new CreateTableAsSelectStmt.CtasParams(new CreateTableStmt(tbl_def),
         select_stmt, partition_cols);
   :}
   ;
 
 create_tbl_stmt ::=
-  tbl_def_without_col_defs:tbl_def
+  KW_CREATE tbl_def_without_col_defs:tbl_def
   tbl_options:options
   {:
     tbl_def.setOptions(options);
     RESULT = new CreateTableStmt(tbl_def);
   :}
-  | tbl_def_without_col_defs:tbl_def
+  | KW_CREATE tbl_def_without_col_defs:tbl_def
     // If "opt_tbl_data_layout" were used instead so that this rule could be combined with
     // the rule above, there would be a conflict with the first rule in
     // "create_tbl_as_select_stmt".
@@ -1154,7 +1173,7 @@ create_tbl_stmt ::=
     create_tbl_stmt.getPartitionColumnDefs().addAll(partition_column_defs);
     RESULT = create_tbl_stmt;
   :}
-  | tbl_def_with_col_defs:tbl_def
+  | KW_CREATE tbl_def_with_col_defs:tbl_def
     opt_tbl_data_layout:data_layout
     tbl_options:options
   {:
@@ -1163,7 +1182,7 @@ create_tbl_stmt ::=
     tbl_def.setOptions(options);
     RESULT = new CreateTableStmt(tbl_def);
   :}
-  | tbl_def_with_col_defs:tbl_def
+  | KW_CREATE tbl_def_with_col_defs:tbl_def
     KW_PRODUCED KW_BY KW_DATA source_ident:is_source_id ident_or_default:data_src_name
     opt_init_string_val:init_string
     opt_comment_val:comment
@@ -1177,7 +1196,7 @@ create_tbl_stmt ::=
     RESULT = new CreateTableDataSrcStmt(new CreateTableStmt(tbl_def),
         data_src_name, init_string);
   :}
-  | tbl_def_without_col_defs:tbl_def
+  | KW_CREATE tbl_def_without_col_defs:tbl_def
     KW_LIKE file_format_val:schema_file_format
     STRING_LITERAL:schema_location
     opt_tbl_data_layout:data_layout
@@ -1197,7 +1216,7 @@ create_tbl_stmt ::=
 // TODO: Refactor the CREATE TABLE statements to improve the grammar and the way we
 // handle table options.
 create_tbl_like_stmt ::=
-  tbl_def_without_col_defs:tbl_def
+  KW_CREATE tbl_def_without_col_defs:tbl_def
   KW_LIKE table_name:other_table
   opt_comment_val:comment
   file_format_create_table_val:file_format location_val:location
@@ -1207,7 +1226,7 @@ create_tbl_like_stmt ::=
   :}
   // This extra production is necessary since without it the parser will not be able to
   // parse "CREATE TABLE A LIKE B".
-  | tbl_def_without_col_defs:tbl_def
+  | KW_CREATE tbl_def_without_col_defs:tbl_def
     opt_sort_cols:sort_cols
     KW_LIKE table_name:other_table
     opt_comment_val:comment
@@ -1221,8 +1240,8 @@ create_tbl_like_stmt ::=
 // Used for creating tables where the schema is inferred externally, e.g., from an Avro
 // schema, Kudu table or query statement.
 tbl_def_without_col_defs ::=
-  KW_CREATE external_val:external KW_TABLE if_not_exists_val:if_not_exists
-  table_name:table
+  external_val:external KW_TABLE
+  if_not_exists_val:if_not_exists table_name:table
   {: RESULT = new TableDef(table, external, if_not_exists); :}
   ;
 
@@ -2529,6 +2548,13 @@ opt_outer ::=
   ;
 
 opt_plan_hints ::=
+  plan_hints:hints
+  {: RESULT = hints; :}
+  | /* empty */
+  {: RESULT = Lists.newArrayList(); :}
+  ;
+
+plan_hints ::=
   COMMENTED_PLAN_HINT_START plan_hint_list:hints COMMENTED_PLAN_HINT_END
   {: RESULT = hints; :}
   /* legacy straight_join hint style */
@@ -2537,8 +2563,6 @@ opt_plan_hints ::=
   /* legacy plan-hint style */
   | LBRACKET plan_hint_list:hints RBRACKET
   {: RESULT = hints; :}
-  | /* empty */
-  {: RESULT = Lists.newArrayList(); :}
   ;
 
 plan_hint ::=

http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
index c5c092d..222b99a 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateTableAsSelectStmt.java
@@ -66,23 +66,40 @@ public class CreateTableAsSelectStmt extends StatementBase {
       EnumSet.of(THdfsFileFormat.PARQUET, THdfsFileFormat.TEXT, THdfsFileFormat.KUDU);
 
   /**
+   * Helper class for parsing.
+   * Contains every parameter of the constructor with the exception of hints. This is
+   * needed to keep the production rules that check for optional hints separate from the
+   * rules that check for optional partition info. Merging these independent rules would
+   * make it necessary to create rules for every combination of them.
+   */
+  public static class CtasParams {
+    public CreateTableStmt createStmt;
+    public QueryStmt queryStmt;
+    public List<String> partitionKeys;
+
+    public CtasParams(CreateTableStmt createStmt, QueryStmt queryStmt,
+        List<String> partitionKeys) {
+      this.createStmt = Preconditions.checkNotNull(createStmt);
+      this.queryStmt = Preconditions.checkNotNull(queryStmt);
+      this.partitionKeys = partitionKeys;
+    }
+  }
+
+  /**
    * Builds a CREATE TABLE AS SELECT statement
    */
-  public CreateTableAsSelectStmt(CreateTableStmt createStmt, QueryStmt queryStmt,
-      List<String> partitionKeys) {
-    Preconditions.checkNotNull(queryStmt);
-    Preconditions.checkNotNull(createStmt);
-    createStmt_ = createStmt;
-    partitionKeys_ = partitionKeys;
+  public CreateTableAsSelectStmt(CtasParams params, List<PlanHint> planHints) {
+    createStmt_ = params.createStmt;
+    partitionKeys_ = params.partitionKeys;
     List<PartitionKeyValue> pkvs = null;
-    if (partitionKeys != null) {
+    if (partitionKeys_ != null) {
       pkvs = Lists.newArrayList();
-      for (String key: partitionKeys) {
+      for (String key: partitionKeys_) {
         pkvs.add(new PartitionKeyValue(key, null));
       }
     }
-    insertStmt_ = InsertStmt.createInsert(
-        null, createStmt.getTblName(), false, pkvs, null, null, queryStmt, null);
+    insertStmt_ = InsertStmt.createInsert(null, createStmt_.getTblName(), false, pkvs,
+        planHints, null, params.queryStmt, null);
   }
 
   public QueryStmt getQueryStmt() { return insertStmt_.getQueryStmt(); }

http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
----------------------------------------------------------------------
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 ee4ed0e..56f81e3 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -1830,6 +1830,34 @@ public class AnalyzeDDLTest extends FrontendTestBase {
   }
 
   @Test
+  public void TestCreateTableAsSelectWithHints() throws AnalysisException {
+    // Test if CTAS hints are analyzed correctly and that conflicting hints
+    // result in error.
+    // The tests here are minimal, because other tests already cover this logic:
+    // - ParserTests#TestPlanHints tests if hints are set correctly during parsing.
+    // - AnalyzeStmtsTest#TestInsertHints tests the analyzes of insert hints, which
+    //   is the same as the analyzes of CTAS hints.
+    for (String[] hintStyle: hintStyles_) {
+      String prefix = hintStyle[0];
+      String suffix = hintStyle[1];
+      // Test plan hints for partitioned Hdfs tables.
+      AnalyzesOk(String.format("create %sshuffle%s table t " +
+          "partitioned by (year, month) as select * from functional.alltypes",
+          prefix, suffix));
+      // Warn on unrecognized hints.
+      AnalyzesOk(String.format("create %sbadhint%s table t " +
+          "partitioned by (year, month) as select * from functional.alltypes",
+          prefix, suffix),
+          "INSERT hint not recognized: badhint");
+      // Conflicting plan hints.
+      AnalysisError(String.format("create %sshuffle,noshuffle%s table t " +
+          "partitioned by (year, month) as " +
+          "select * from functional.alltypes", prefix, suffix),
+          "Conflicting INSERT hints: shuffle and noshuffle");
+    }
+  }
+
+  @Test
   public void TestCreateTableLike() throws AnalysisException {
     AnalyzesOk("create table if not exists functional.new_tbl like functional.alltypes");
     AnalyzesOk("create table functional.like_view like functional.view_view");

http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
index cdcb9bc..ab66482 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeStmtsTest.java
@@ -1737,12 +1737,7 @@ public class AnalyzeStmtsTest extends AnalyzerTest {
 
   @Test
   public void TestSelectListHints() throws AnalysisException {
-    String[][] hintStyles = new String[][] {
-        new String[] { "/* +", "*/" }, // traditional commented hint
-        new String[] { "\n-- +", "\n" }, // eol commented hint
-        new String[] { "", "" } // without surrounding characters
-    };
-    for (String[] hintStyle: hintStyles) {
+    for (String[] hintStyle: hintStyles_) {
       String prefix = hintStyle[0];
       String suffix = hintStyle[1];
       AnalyzesOk(String.format(

http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
----------------------------------------------------------------------
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 8dd4898..178539f 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -300,11 +300,19 @@ public class ParserTest extends FrontendTestBase {
     assertEquals(Lists.newArrayList(expectedHints), actualHints);
   }
 
-  private void VerifyHints(List<PlanHint> actualHints, String... expectedHints) {
-    List<String> stringHints = Lists.newArrayList();
-    for (PlanHint hint: actualHints) stringHints.add(hint.toString());
-    if (stringHints.isEmpty()) stringHints = Lists.newArrayList((String) null);
-    assertEquals(Lists.newArrayList(expectedHints), stringHints);
+  /**
+   * Creates an insert into, an insert overwrite, and a CTAS statement with
+   * the given hints and checks that the parsed hints are the same as the expected hints.
+   */
+  private void TestInsertAndCtasHints(String insertPart, String ctasPart,
+        String[] hintStyle, String hints, String... expectedHints) {
+    String hintsPart = hintStyle[0] + hints + hintStyle[1];
+    TestInsertStmtHints(String.format("insert %%s into %s %%s select * from t",
+        insertPart), hintsPart, expectedHints);
+    TestInsertStmtHints(String.format("insert %%s overwrite %s %%s select * from t",
+        insertPart), hintsPart, expectedHints);
+    TestCtasHints(String.format("create %s table %s as select * from t",
+        hintsPart, ctasPart), expectedHints);
   }
 
   /**
@@ -313,8 +321,8 @@ public class ParserTest extends FrontendTestBase {
    */
   private void TestInsertStmtHints(String pattern, String hint, String... expectedHints) {
     for (InsertStmt.HintLocation loc: InsertStmt.HintLocation.values()) {
-      VerifyHints(((InsertStmt) ParsesOk(InjectInsertHint(pattern, hint, loc)))
-          .getPlanHints(), expectedHints);
+      InsertStmt insertStmt = (InsertStmt) ParsesOk(InjectInsertHint(pattern, hint, loc));
+      assertEquals(expectedHints, HintsToStrings(insertStmt.getPlanHints()));
     }
   }
 
@@ -328,20 +336,28 @@ public class ParserTest extends FrontendTestBase {
     }
   }
 
+  /**
+   * Parses stmt and checks that the CTAS hints stmt are the expected hints.
+   */
+  private void TestCtasHints(String stmt, String... expectedHints) {
+    CreateTableAsSelectStmt ctasStmt = (CreateTableAsSelectStmt) ParsesOk(stmt);
+    assertEquals(expectedHints, HintsToStrings(ctasStmt.getInsertStmt().getPlanHints()));
+  }
+
+  static private String[] HintsToStrings(List<PlanHint> hints) {
+    if (hints.isEmpty()) return new String[] { null };
+    String[] hintAsStrings = new String[hints.size()];
+    for (int i = 0; i < hints.size(); ++i) hintAsStrings[i] = hints.get(i).toString();
+    return hintAsStrings;
+  }
+
   @Test
   public void TestPlanHints() {
-    // All plan-hint styles embed a comma-separated list of hints.
-    String[][] hintStyles = new String[][] {
-        new String[] { "/* +", "*/" }, // traditional commented hint
-        new String[] { "-- +", "\n" }, // eol commented hint
-        new String[] { "\n-- +", "\n" }, // eol commented hint
-        new String[] { "[", "]" } // legacy style
-    };
     String[][] commentStyles = new String[][] {
         new String[] { "/*", "*/" }, // traditional comment
         new String[] { "--", "\n" } // eol comment
     };
-    for (String[] hintStyle: hintStyles) {
+    for (String[] hintStyle: hintStyles_) {
       String prefix = hintStyle[0];
       String suffix = hintStyle[1];
       // Test join hints.
@@ -389,17 +405,14 @@ public class ParserTest extends FrontendTestBase {
               "join %sshuffle%s functional.alltypes e using(string_col)",
               suffix, suffix, suffix, suffix, prefix, "", "", ""));
 
-      // Test insert hints.
-      TestInsertStmtHints("insert %s into t %s select * from t",
-           String.format("%snoshuffle%s", prefix, suffix), "noshuffle");
-      TestInsertStmtHints("insert %s overwrite t %s select * from t",
-           String.format("%snoshuffle%s", prefix, suffix), "noshuffle");
-      TestInsertStmtHints("insert %s into t partition(x, y) %s select * from t",
-           String.format("%snoshuffle%s", prefix, suffix), "noshuffle");
-      TestInsertStmtHints("insert %s into t(a, b) partition(x, y) %s select * from t",
-           String.format("%sshuffle%s", prefix, suffix), "shuffle");
-      TestInsertStmtHints("insert %s overwrite t(a, b) partition(x, y) %s select * from t",
-           String.format("%sfoo,bar,baz%s", prefix, suffix), "foo", "bar", "baz");
+      // Test insert/CTAS hints.
+      TestInsertAndCtasHints("t", "t", hintStyle, "noshuffle", "noshuffle");
+      TestInsertAndCtasHints("t partition(x, y)", "t partitioned by(x, y)",
+          hintStyle, "noshuffle", "noshuffle");
+      TestInsertAndCtasHints("t(a, b) partition(x, y)", "t partitioned by(x, y)",
+          hintStyle, "shuffle", "shuffle");
+      TestInsertAndCtasHints("t(a, b) partition(x, y)", "t partitioned by(x, y)",
+          hintStyle, "foo,bar,baz", "foo", "bar", "baz");
 
       // Test upsert hints.
       TestInsertStmtHints("upsert %s into t %s select * from t",
@@ -488,14 +501,29 @@ public class ParserTest extends FrontendTestBase {
       ParserErrorOnInsertStmtHints("insert %s into t %s select * from t",
            String.format("%shint_with_args(  a  ,  , ,,, b  )%s", prefix, suffix));
 
-      // Negative tests for hints cannot be specified at the both avilable locations.
+      TestInsertAndCtasHints("t", "t",
+          hintStyle, "hint_with_args(a)", "hint_with_args(a)");
+      TestInsertAndCtasHints("t", "t",
+          hintStyle, "clustered,shuffle,hint_with_args(a)",
+          "clustered", "shuffle", "hint_with_args(a)");
+      TestInsertAndCtasHints("t", "t",
+          hintStyle, "hint_with_args(a,b)", "hint_with_args(a,b)");
+      TestInsertAndCtasHints("t", "t",
+          hintStyle, "hint_with_args(a  , b)", "hint_with_args(a,b)");
+
+      ParserErrorOnInsertStmtHints("insert %s into t %s select * from t",
+          String.format("%shint_with_args(  a  ,  , ,,, b  )%s", prefix, suffix));
+      ParserError(String.format(
+         "create table t %shint_with_args(  a  ,  , ,,, b  )%s as select * from t",
+         prefix, suffix));
+
+      // Negative tests for hints cannot be specified at the both available locations.
       ParserError(String.format("insert %s into t %s select * from t",
            String.format("%sshuffle%s", prefix, suffix),
            String.format("%sclustered%s", prefix, suffix)));
       ParserError(String.format("upsert %s into t %s select * from t",
            String.format("%sshuffle%s", prefix, suffix),
            String.format("%sclustered%s", prefix, suffix)));
-
     }
     // No "+" at the beginning so the comment is not recognized as a hint.
     TestJoinHints("select * from functional.alltypes a join /* comment */" +
@@ -504,6 +532,8 @@ public class ParserTest extends FrontendTestBase {
         (String) null);
     TestInsertStmtHints("insert %s into t(a, b) partition(x, y) %s select 1",
         "/* comment */", (String) null);
+    TestCtasHints("create /* comment */ table t partitioned by (x, y) as select 1",
+        (String) null);
     TestSelectListHints("select /* -- +straight_join */ * from functional.alltypes",
         (String) null);
     TestSelectListHints("select /* abcdef +straight_join */ * from functional.alltypes",

http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
index a813cce..760ea59 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ToSqlTest.java
@@ -546,12 +546,7 @@ public class ToSqlTest extends FrontendTestBase {
    */
   @Test
   public void planHintsTest() {
-    String[][] hintStyles = new String[][] {
-        new String[] { "/* +", "*/" }, // traditional commented hint
-        new String[] { "\n-- +", "\n" }, // eol commented hint
-        new String[] { "[", "]" } // legacy style
-    };
-    for (String[] hintStyle: hintStyles) {
+    for (String[] hintStyle: hintStyles_) {
       String prefix = hintStyle[0];
       String suffix = hintStyle[1];
 

http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java b/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
index 9be3343..28dc710 100644
--- a/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
+++ b/fe/src/test/java/org/apache/impala/common/FrontendTestBase.java
@@ -84,6 +84,11 @@ public class FrontendTestBase {
   // Test-local list of test databases and tables. These are cleaned up in @After.
   protected final List<Db> testDbs_ = Lists.newArrayList();
   protected final List<Table> testTables_ = Lists.newArrayList();
+  protected final String[][] hintStyles_ = new String[][] {
+      new String[] { "/* +", "*/" }, // traditional commented hint
+      new String[] { "\n-- +", "\n" }, // eol commented hint
+      new String[] { "[", "]" } // legacy style
+  };
 
   @BeforeClass
   public static void setUp() throws Exception {

http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/testdata/workloads/functional-planner/queries/PlannerTest/ddl.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/ddl.test b/testdata/workloads/functional-planner/queries/PlannerTest/ddl.test
index 0129ad1..74c7e5f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/ddl.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/ddl.test
@@ -204,6 +204,37 @@ WRITE TO HDFS [default.t, OVERWRITE=false, PARTITION-KEYS=(year,month)]
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ====
+# IMPALA-4167: if no (no)shuffle hint is given for CTAS into partitioned HDFS table, then
+# Impala is free to decide whether to add an exchange node or not. In this example, the
+# planner decides to shuffle. The aim of this test is to have a reference query where
+# noshuffle hint would lead to a different plan.
+# Note that noclustered hint is added to ensure consistent plans on Impala 2.x and 3.x,
+# because IMPALA-5293 changed clustered to be the default on 3.x.
+create /*+ noclustered */table t partitioned by (year, month) as
+select * from functional.alltypes
+---- DISTRIBUTEDPLAN
+WRITE TO HDFS [default.t, OVERWRITE=false, PARTITION-KEYS=(functional.alltypes.year,functional.alltypes.month)]
+|  partitions=24
+|
+01:EXCHANGE [HASH(functional.alltypes.year,functional.alltypes.month)]
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====
+# IMPALA-4167: non-shuffled CTAS into partitioned table has no exchange node before write.
+# Note that plan hint tests for CTAS are minimal by design, as this logic is covered well
+# in insert.test.
+# Note that noclustered hint is added to ensure consistent plans on Impala 2.x and 3.x,
+# because IMPALA-5293 changed clustered to be the default on 3.x.
+create /*+ noshuffle, noclustered */ table t partitioned by (year, month) as
+select * from functional.alltypes
+---- DISTRIBUTEDPLAN
+WRITE TO HDFS [default.t, OVERWRITE=false, PARTITION-KEYS=(functional.alltypes.year,functional.alltypes.month)]
+|  partitions=24
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====
 # CTAS with more complex select query
 create table t partitioned by (c_nationkey) sort by (c_custkey) as
 select c_custkey, max(o_totalprice) as maxprice, c_nationkey

http://git-wip-us.apache.org/repos/asf/impala/blob/07d9f71a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
index 835c41b..d506ae7 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
@@ -426,6 +426,27 @@ INSERT INTO KUDU [functional_kudu.alltypes]
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ====
+# Hint - noclustered should remove the sort node from CTAS.
+create /* +noclustered */ table t
+primary key(id) partition by hash(id) partitions 3 stored as kudu as
+select * from functional.alltypes;
+---- DISTRIBUTEDPLAN
+INSERT INTO KUDU [default.t]
+|
+01:EXCHANGE [KUDU(KuduPartition(functional.alltypes.id))]
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====
+create /* +noclustered,noshuffle */ table t
+primary key(id) partition by hash(id) partitions 3 stored as kudu as
+select * from functional.alltypes;
+---- DISTRIBUTEDPLAN
+INSERT INTO KUDU [default.t]
+|
+00:SCAN HDFS [functional.alltypes]
+   partitions=24/24 files=24 size=478.45KB
+====
 # IMPALA-5602: If a query contains predicates that are all pushed to kudu and there is a
 # limit, then the query should not incorrectly run with 'small query' optimization.
 select * from functional_kudu.alltypesagg where tinyint_col = 9 limit 10;


[02/15] impala git commit: IMPALA-4835: Part 3: switch I/O buffers to buffer pool

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test b/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
index 146524a..b10c5cd 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/sort-expr-materialization.test
@@ -2,7 +2,7 @@
 select * from functional.alltypes order by random()
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.00MB
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -19,14 +19,14 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=7300
 ====
 # sort on a deterministic expr that exceeds the cost threshold
 select * from functional.alltypes order by abs(id) + abs(id)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.00MB
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -43,14 +43,14 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=7300
 ====
 # sort on a deterministic expr that doesn't exceed the cost threshold
 select * from functional.alltypes order by tinyint_col + 1
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.00MB
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -66,7 +66,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=7300
 ====
 # sort on multiple exprs, subset is materialized
@@ -74,7 +74,7 @@ select * from functional.alltypes
 order by dayofweek(timestamp_col), true, id + 1, string_col = date_string_col, id = tinyint_col
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.00MB
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -91,7 +91,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=7300
 ====
 # expensive analytic order by expr gets materialized
@@ -99,7 +99,7 @@ select last_value(id) over (order by to_date(timestamp_col), bool_col is null)
 from functional.alltypes
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=16.00MB
+|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=16.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -123,7 +123,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=21B cardinality=7300
 ====
 # expensive order by expr in top-n gets materialized
@@ -131,7 +131,7 @@ select id from functional.alltypes order by string_col like 'a.*b', id * bigint_
 regexp_replace(string_col, 'a.*b', 'c') limit 10
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=128.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -148,14 +148,14 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=29B cardinality=7300
 ====
 # sort on udf, gets materialized
 select * from functional.alltypes order by TestFn(double_col)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.00MB
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -172,14 +172,14 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=7300
 ====
 # sort expr contains SlotRefs that don't need to be materialized separately
 select concat(date_string_col, string_col) c from functional.alltypes order by c
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.00MB
+|  Per-Host Resources: mem-estimate=140.00MB mem-reservation=12.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -196,6 +196,6 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=41B cardinality=7300
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
index 03273bc..f57e578 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/spillable-buffer-sizing.test
@@ -3,7 +3,7 @@ select straight_join *
 from tpch_parquet.customer
     inner join tpch_parquet.nation on c_nationkey = n_nationkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=2.94MB
+Max Per-Host Resource Reservation: Memory=18.97MB
 Per-Host Resource Estimates: Memory=42.94MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -16,7 +16,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=355B cardinality=150000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-Per-Host Resources: mem-estimate=26.94MB mem-reservation=2.94MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=26.94MB mem-reservation=18.94MB runtime-filters-memory=1.00MB
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c_nationkey = n_nationkey
 |  fk/pk conjuncts: c_nationkey = n_nationkey
@@ -29,14 +29,14 @@ Per-Host Resources: mem-estimate=26.94MB mem-reservation=2.94MB runtime-filters-
 |  |  tuple-ids=1 row-size=117B cardinality=25
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB
 |  01:SCAN HDFS [tpch_parquet.nation, RANDOM]
 |     partitions=1/1 files=1 size=2.74KB
 |     stored statistics:
 |       table: rows=25 size=2.74KB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=16.00MB mem-reservation=0B
+|     mem-estimate=16.00MB mem-reservation=32.00KB
 |     tuple-ids=1 row-size=117B cardinality=25
 |
 00:SCAN HDFS [tpch_parquet.customer, RANDOM]
@@ -46,10 +46,10 @@ Per-Host Resources: mem-estimate=26.94MB mem-reservation=2.94MB runtime-filters-
      table: rows=150000 size=12.31MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=24.00MB mem-reservation=0B
+   mem-estimate=24.00MB mem-reservation=16.00MB
    tuple-ids=0 row-size=238B cardinality=150000
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=5.88MB
+Max Per-Host Resource Reservation: Memory=37.94MB
 Per-Host Resource Estimates: Memory=85.88MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -62,7 +62,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=355B cardinality=150000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-Per-Host Resources: mem-estimate=53.88MB mem-reservation=5.88MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=53.88MB mem-reservation=37.88MB runtime-filters-memory=1.00MB
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: c_nationkey = n_nationkey
@@ -83,14 +83,14 @@ Per-Host Resources: mem-estimate=53.88MB mem-reservation=5.88MB runtime-filters-
 |  |  tuple-ids=1 row-size=117B cardinality=25
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=64.00KB
 |  01:SCAN HDFS [tpch_parquet.nation, RANDOM]
 |     partitions=1/1 files=1 size=2.74KB
 |     stored statistics:
 |       table: rows=25 size=2.74KB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=16.00MB mem-reservation=0B
+|     mem-estimate=16.00MB mem-reservation=32.00KB
 |     tuple-ids=1 row-size=117B cardinality=25
 |
 00:SCAN HDFS [tpch_parquet.customer, RANDOM]
@@ -100,7 +100,7 @@ Per-Host Resources: mem-estimate=53.88MB mem-reservation=5.88MB runtime-filters-
      table: rows=150000 size=12.31MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=24.00MB mem-reservation=0B
+   mem-estimate=24.00MB mem-reservation=16.00MB
    tuple-ids=0 row-size=238B cardinality=150000
 ====
 # Join with large build side - should use default-sized buffers.
@@ -108,8 +108,8 @@ select straight_join *
 from tpch_parquet.lineitem
     left join tpch_parquet.orders on l_orderkey = o_orderkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=34.00MB
-Per-Host Resource Estimates: Memory=420.41MB
+Max Per-Host Resource Reservation: Memory=154.00MB
+Per-Host Resource Estimates: Memory=428.41MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -121,7 +121,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1N row-size=454B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=380.41MB mem-reservation=34.00MB
+Per-Host Resources: mem-estimate=380.41MB mem-reservation=106.00MB
 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
@@ -133,27 +133,27 @@ Per-Host Resources: mem-estimate=380.41MB mem-reservation=34.00MB
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=48.00MB
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     partitions=1/1 files=2 size=54.07MB
 |     stored statistics:
 |       table: rows=1500000 size=54.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=40.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=48.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=68.00MB
-Per-Host Resource Estimates: Memory=840.83MB
+Max Per-Host Resource Reservation: Memory=308.00MB
+Per-Host Resource Estimates: Memory=856.83MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -165,7 +165,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1N row-size=454B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=760.83MB mem-reservation=68.00MB
+Per-Host Resources: mem-estimate=760.83MB mem-reservation=212.00MB
 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: l_orderkey = o_orderkey
@@ -185,23 +185,23 @@ Per-Host Resources: mem-estimate=760.83MB mem-reservation=68.00MB
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=96.00MB mem-reservation=96.00MB
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     partitions=1/1 files=2 size=54.07MB
 |     stored statistics:
 |       table: rows=1500000 size=54.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=40.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=48.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Shuffle join with mid-sized input.
@@ -209,8 +209,8 @@ select straight_join *
 from tpch_parquet.orders
     join /*+shuffle*/ tpch_parquet.customer on o_custkey = c_custkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=36.00MB
-Per-Host Resource Estimates: Memory=100.00MB
+Max Per-Host Resource Reservation: Memory=100.00MB
+Per-Host Resource Estimates: Memory=108.00MB
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -235,14 +235,14 @@ Per-Host Resources: mem-estimate=35.00MB mem-reservation=35.00MB runtime-filters
 |  |  tuple-ids=1 row-size=238B cardinality=150000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=24.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=24.00MB mem-reservation=16.00MB
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
 |     partitions=1/1 files=1 size=12.31MB
 |     stored statistics:
 |       table: rows=150000 size=12.31MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=24.00MB mem-reservation=0B
+|     mem-estimate=24.00MB mem-reservation=16.00MB
 |     tuple-ids=1 row-size=238B cardinality=150000
 |
 03:EXCHANGE [HASH(o_custkey)]
@@ -250,7 +250,7 @@ Per-Host Resources: mem-estimate=35.00MB mem-reservation=35.00MB runtime-filters
 |  tuple-ids=0 row-size=191B cardinality=1500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-Per-Host Resources: mem-estimate=41.00MB mem-reservation=1.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=49.00MB mem-reservation=49.00MB runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
    partitions=1/1 files=2 size=54.07MB
    runtime filters: RF000[bloom] -> o_custkey
@@ -258,11 +258,11 @@ Per-Host Resources: mem-estimate=41.00MB mem-reservation=1.00MB runtime-filters-
      table: rows=1500000 size=54.07MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=40.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=48.00MB
    tuple-ids=0 row-size=191B cardinality=1500000
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=38.00MB
-Per-Host Resource Estimates: Memory=166.00MB
+Max Per-Host Resource Reservation: Memory=166.00MB
+Per-Host Resource Estimates: Memory=182.00MB
 
 F03:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -295,14 +295,14 @@ Per-Host Resources: mem-estimate=36.00MB mem-reservation=36.00MB runtime-filters
 |  |  tuple-ids=1 row-size=238B cardinality=150000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00MB
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
 |     partitions=1/1 files=1 size=12.31MB
 |     stored statistics:
 |       table: rows=150000 size=12.31MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=24.00MB mem-reservation=0B
+|     mem-estimate=24.00MB mem-reservation=16.00MB
 |     tuple-ids=1 row-size=238B cardinality=150000
 |
 03:EXCHANGE [HASH(o_custkey)]
@@ -310,7 +310,7 @@ Per-Host Resources: mem-estimate=36.00MB mem-reservation=36.00MB runtime-filters
 |  tuple-ids=0 row-size=191B cardinality=1500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-Per-Host Resources: mem-estimate=82.00MB mem-reservation=2.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=98.00MB mem-reservation=98.00MB runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
    partitions=1/1 files=2 size=54.07MB
    runtime filters: RF000[bloom] -> o_custkey
@@ -318,7 +318,7 @@ Per-Host Resources: mem-estimate=82.00MB mem-reservation=2.00MB runtime-filters-
      table: rows=1500000 size=54.07MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=40.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=48.00MB
    tuple-ids=0 row-size=191B cardinality=1500000
 ====
 # Broadcast join with mid-sized input - should use larger buffers than shuffle join.
@@ -326,8 +326,8 @@ select straight_join *
 from tpch_parquet.orders
     join /*+broadcast*/ tpch_parquet.customer on o_custkey = c_custkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=35.00MB
-Per-Host Resource Estimates: Memory=102.38MB
+Max Per-Host Resource Reservation: Memory=99.00MB
+Per-Host Resource Estimates: Memory=110.38MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -339,7 +339,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=428B cardinality=1500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-Per-Host Resources: mem-estimate=78.38MB mem-reservation=35.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=86.38MB mem-reservation=83.00MB runtime-filters-memory=1.00MB
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: o_custkey = c_custkey
 |  fk/pk conjuncts: o_custkey = c_custkey
@@ -352,14 +352,14 @@ Per-Host Resources: mem-estimate=78.38MB mem-reservation=35.00MB runtime-filters
 |  |  tuple-ids=1 row-size=238B cardinality=150000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=24.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=24.00MB mem-reservation=16.00MB
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
 |     partitions=1/1 files=1 size=12.31MB
 |     stored statistics:
 |       table: rows=150000 size=12.31MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=24.00MB mem-reservation=0B
+|     mem-estimate=24.00MB mem-reservation=16.00MB
 |     tuple-ids=1 row-size=238B cardinality=150000
 |
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
@@ -369,11 +369,11 @@ Per-Host Resources: mem-estimate=78.38MB mem-reservation=35.00MB runtime-filters
      table: rows=1500000 size=54.07MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=40.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=48.00MB
    tuple-ids=0 row-size=191B cardinality=1500000
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=70.00MB
-Per-Host Resource Estimates: Memory=204.76MB
+Max Per-Host Resource Reservation: Memory=198.00MB
+Per-Host Resource Estimates: Memory=220.76MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -385,7 +385,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=428B cardinality=1500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-Per-Host Resources: mem-estimate=156.76MB mem-reservation=70.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=172.76MB mem-reservation=166.00MB runtime-filters-memory=1.00MB
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: o_custkey = c_custkey
@@ -406,14 +406,14 @@ Per-Host Resources: mem-estimate=156.76MB mem-reservation=70.00MB runtime-filter
 |  |  tuple-ids=1 row-size=238B cardinality=150000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00MB
 |  01:SCAN HDFS [tpch_parquet.customer, RANDOM]
 |     partitions=1/1 files=1 size=12.31MB
 |     stored statistics:
 |       table: rows=150000 size=12.31MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=24.00MB mem-reservation=0B
+|     mem-estimate=24.00MB mem-reservation=16.00MB
 |     tuple-ids=1 row-size=238B cardinality=150000
 |
 00:SCAN HDFS [tpch_parquet.orders, RANDOM]
@@ -423,7 +423,7 @@ Per-Host Resources: mem-estimate=156.76MB mem-reservation=70.00MB runtime-filter
      table: rows=1500000 size=54.07MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=40.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=48.00MB
    tuple-ids=0 row-size=191B cardinality=1500000
 ====
 # Join with no stats for right input - should use default buffers.
@@ -431,7 +431,7 @@ select straight_join *
 from functional_parquet.alltypes
     left join functional_parquet.alltypestiny on alltypes.id = alltypestiny.id
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=34.00MB
+Max Per-Host Resource Reservation: Memory=34.17MB
 Per-Host Resource Estimates: Memory=2.03GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes, functional_parquet.alltypestiny
@@ -446,7 +446,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1N row-size=176B cardinality=unavailable
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.00MB
+Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.09MB
 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
 |  hash predicates: alltypes.id = alltypestiny.id
 |  fk/pk conjuncts: assumed fk/pk
@@ -458,7 +458,7 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.00MB
 |  |  tuple-ids=1 row-size=88B cardinality=unavailable
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=88.00KB
 |  01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
 |     partitions=4/4 files=4 size=9.75KB
 |     stored statistics:
@@ -466,7 +466,7 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.00MB
 |       partitions: 0/4 rows=unavailable
 |       columns: unavailable
 |     extrapolated-rows=disabled
-|     mem-estimate=16.00MB mem-reservation=0B
+|     mem-estimate=16.00MB mem-reservation=88.00KB
 |     tuple-ids=1 row-size=88B cardinality=unavailable
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
@@ -476,10 +476,10 @@ Per-Host Resources: mem-estimate=2.02GB mem-reservation=34.00MB
      partitions: 0/24 rows=unavailable
      columns missing stats: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=88.00KB
    tuple-ids=0 row-size=88B cardinality=unavailable
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=68.00MB
+Max Per-Host Resource Reservation: Memory=68.34MB
 Per-Host Resource Estimates: Memory=4.06GB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypestiny
@@ -494,7 +494,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1N row-size=176B cardinality=unavailable
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=4.03GB mem-reservation=68.00MB
+Per-Host Resources: mem-estimate=4.03GB mem-reservation=68.17MB
 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
 |  hash-table-id=00
 |  hash predicates: alltypes.id = alltypestiny.id
@@ -514,7 +514,7 @@ Per-Host Resources: mem-estimate=4.03GB mem-reservation=68.00MB
 |  |  tuple-ids=1 row-size=88B cardinality=unavailable
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=176.00KB
 |  01:SCAN HDFS [functional_parquet.alltypestiny, RANDOM]
 |     partitions=4/4 files=4 size=9.75KB
 |     stored statistics:
@@ -522,7 +522,7 @@ Per-Host Resources: mem-estimate=4.03GB mem-reservation=68.00MB
 |       partitions: 0/4 rows=unavailable
 |       columns: unavailable
 |     extrapolated-rows=disabled
-|     mem-estimate=16.00MB mem-reservation=0B
+|     mem-estimate=16.00MB mem-reservation=88.00KB
 |     tuple-ids=1 row-size=88B cardinality=unavailable
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
@@ -532,7 +532,7 @@ Per-Host Resources: mem-estimate=4.03GB mem-reservation=68.00MB
      partitions: 0/24 rows=unavailable
      columns missing stats: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=88.00KB
    tuple-ids=0 row-size=88B cardinality=unavailable
 ====
 # Low NDV aggregation - should scale down buffers to minimum.
@@ -540,7 +540,7 @@ select c_nationkey, avg(c_acctbal)
 from tpch_parquet.customer
 group by c_nationkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=3.94MB
+Max Per-Host Resource Reservation: Memory=19.94MB
 Per-Host Resource Estimates: Memory=44.00MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -565,7 +565,7 @@ Per-Host Resources: mem-estimate=10.00MB mem-reservation=1.94MB
 |  tuple-ids=1 row-size=10B cardinality=25
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-Per-Host Resources: mem-estimate=34.00MB mem-reservation=2.00MB
+Per-Host Resources: mem-estimate=34.00MB mem-reservation=18.00MB
 01:AGGREGATE [STREAMING]
 |  output: avg(c_acctbal)
 |  group by: c_nationkey
@@ -578,10 +578,10 @@ Per-Host Resources: mem-estimate=34.00MB mem-reservation=2.00MB
      table: rows=150000 size=12.31MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=24.00MB mem-reservation=0B
+   mem-estimate=24.00MB mem-reservation=16.00MB
    tuple-ids=0 row-size=10B cardinality=150000
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=7.88MB
+Max Per-Host Resource Reservation: Memory=39.88MB
 Per-Host Resource Estimates: Memory=88.00MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -606,7 +606,7 @@ Per-Host Resources: mem-estimate=20.00MB mem-reservation=3.88MB
 |  tuple-ids=1 row-size=10B cardinality=25
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=2
-Per-Host Resources: mem-estimate=68.00MB mem-reservation=4.00MB
+Per-Host Resources: mem-estimate=68.00MB mem-reservation=36.00MB
 01:AGGREGATE [STREAMING]
 |  output: avg(c_acctbal)
 |  group by: c_nationkey
@@ -619,7 +619,7 @@ Per-Host Resources: mem-estimate=68.00MB mem-reservation=4.00MB
      table: rows=150000 size=12.31MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=24.00MB mem-reservation=0B
+   mem-estimate=24.00MB mem-reservation=16.00MB
    tuple-ids=0 row-size=10B cardinality=150000
 ====
 # Mid NDV aggregation - should scale down buffers to intermediate size.
@@ -629,7 +629,7 @@ from tpch_parquet.lineitem
 group by 1, 2
 having count(*) = 1
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=87.00MB
+Max Per-Host Resource Reservation: Memory=111.00MB
 Per-Host Resource Estimates: Memory=227.12MB
 
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -674,14 +674,14 @@ Per-Host Resources: mem-estimate=72.12MB mem-reservation=52.00MB runtime-filters
 |  |  tuple-ids=1 row-size=25B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=16.00MB
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     partitions=1/1 files=2 size=54.07MB
 |     stored statistics:
 |       table: rows=1500000 size=54.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=40.00MB mem-reservation=0B
+|     mem-estimate=40.00MB mem-reservation=16.00MB
 |     tuple-ids=1 row-size=25B cardinality=1500000
 |
 04:EXCHANGE [HASH(l_orderkey)]
@@ -689,18 +689,18 @@ Per-Host Resources: mem-estimate=72.12MB mem-reservation=52.00MB runtime-filters
 |  tuple-ids=0 row-size=8B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=81.00MB mem-reservation=1.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=81.00MB mem-reservation=9.00MB runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=123.00MB
+Max Per-Host Resource Reservation: Memory=171.00MB
 Per-Host Resource Estimates: Memory=363.00MB
 
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -753,14 +753,14 @@ Per-Host Resources: mem-estimate=87.00MB mem-reservation=87.00MB runtime-filters
 |  |  tuple-ids=1 row-size=25B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=4
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=32.00MB
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     partitions=1/1 files=2 size=54.07MB
 |     stored statistics:
 |       table: rows=1500000 size=54.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=40.00MB mem-reservation=0B
+|     mem-estimate=40.00MB mem-reservation=16.00MB
 |     tuple-ids=1 row-size=25B cardinality=1500000
 |
 04:EXCHANGE [HASH(l_orderkey)]
@@ -768,22 +768,22 @@ Per-Host Resources: mem-estimate=87.00MB mem-reservation=87.00MB runtime-filters
 |  tuple-ids=0 row-size=8B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=162.00MB mem-reservation=2.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=162.00MB mem-reservation=18.00MB runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ====
 # High NDV aggregation - should use default buffer size.
 select distinct *
 from tpch_parquet.lineitem
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=68.00MB
+Max Per-Host Resource Reservation: Memory=140.00MB
 Per-Host Resource Estimates: Memory=3.31GB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -807,22 +807,22 @@ Per-Host Resources: mem-estimate=1.62GB mem-reservation=34.00MB
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=1.69GB mem-reservation=34.00MB
+Per-Host Resources: mem-estimate=1.69GB mem-reservation=106.00MB
 01:AGGREGATE [STREAMING]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
 |  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=136.00MB
+Max Per-Host Resource Reservation: Memory=280.00MB
 Per-Host Resource Estimates: Memory=6.62GB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -846,19 +846,19 @@ Per-Host Resources: mem-estimate=3.23GB mem-reservation=68.00MB
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=3.39GB mem-reservation=68.00MB
+Per-Host Resources: mem-estimate=3.39GB mem-reservation=212.00MB
 01:AGGREGATE [STREAMING]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
 |  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Aggregation with unknown input - should use default buffer size.
@@ -866,7 +866,7 @@ select string_col, count(*)
 from functional_parquet.alltypestiny
 group by string_col
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=68.00MB
+Max Per-Host Resource Reservation: Memory=68.01MB
 Per-Host Resource Estimates: Memory=272.00MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypestiny
@@ -893,7 +893,7 @@ Per-Host Resources: mem-estimate=128.00MB mem-reservation=34.00MB
 |  tuple-ids=1 row-size=24B cardinality=unavailable
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.00MB
+Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.01MB
 01:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: string_col
@@ -907,10 +907,10 @@ Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.00MB
      partitions: 0/4 rows=unavailable
      columns: unavailable
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=8.00KB
    tuple-ids=0 row-size=16B cardinality=unavailable
 ---- PARALLELPLANS
-Max Per-Host Resource Reservation: Memory=136.00MB
+Max Per-Host Resource Reservation: Memory=136.02MB
 Per-Host Resource Estimates: Memory=544.00MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypestiny
@@ -937,7 +937,7 @@ Per-Host Resources: mem-estimate=256.00MB mem-reservation=68.00MB
 |  tuple-ids=1 row-size=24B cardinality=unavailable
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=6
-Per-Host Resources: mem-estimate=288.00MB mem-reservation=68.00MB
+Per-Host Resources: mem-estimate=288.00MB mem-reservation=68.02MB
 01:AGGREGATE [STREAMING]
 |  output: count(*)
 |  group by: string_col
@@ -951,6 +951,6 @@ Per-Host Resources: mem-estimate=288.00MB mem-reservation=68.00MB
      partitions: 0/4 rows=unavailable
      columns: unavailable
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=8.00KB
    tuple-ids=0 row-size=16B cardinality=unavailable
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test b/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
index 48163ec..1844f11 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tablesample.test
@@ -2,7 +2,7 @@
 select * from functional.alltypes tablesample system(10) repeatable(1234)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -13,14 +13,14 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=730
 ====
 # Sample 50%
 select * from functional.alltypes tablesample system(50) repeatable(1234)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -31,7 +31,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=3650
 ====
 # Sampling and scan predicates. Scan predicates are evaluated after sampling and
@@ -40,7 +40,7 @@ select * from functional.alltypes tablesample system(50) repeatable(1234)
 where id < 10
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=80.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -53,7 +53,7 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled
    parquet dictionary predicates: id < 10
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=365
 ====
 # Partition pruning + sampling. Partition pruning happens before sampling.
@@ -61,7 +61,7 @@ select * from functional.alltypes tablesample system(50) repeatable(1234)
 where year = 2009
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -72,7 +72,7 @@ PLAN-ROOT SINK
      partitions: 12/12 rows=3650
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=48.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=1825
 ====
 # Edge case: sample 0%, no files should be selected
@@ -97,7 +97,7 @@ PLAN-ROOT SINK
 select * from functional.alltypes tablesample system(1) repeatable(1234)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -108,7 +108,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=73
 ====
 # Edge case: sample 1% and prune partitions, at least one file should be selected
@@ -116,7 +116,7 @@ select * from functional.alltypes tablesample system(1) repeatable(1234)
 where year = 2010
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -127,14 +127,14 @@ PLAN-ROOT SINK
      partitions: 12/12 rows=3650
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=37
 ====
 # Edge case: sample 100%, all files should be selected
 select * from functional.alltypes tablesample system (100)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=128.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -145,25 +145,25 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=97B cardinality=7300
 ====
 # Table that has no stats.
 select id from functional_parquet.alltypes tablesample system(10) repeatable(1234)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=8.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=3/24 files=3 size=22.07KB
+   partitions=3/24 files=3 size=22.10KB
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/24 rows=unavailable
      columns: unavailable
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=8.00KB
    tuple-ids=0 row-size=4B cardinality=unavailable
 ====
 # Sampling in a subquery.
@@ -172,7 +172,7 @@ select id from functional.alltypes t1 where exists (
   where t1.id = t2.id)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=162.94MB mem-reservation=2.94MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=162.94MB mem-reservation=2.98MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -189,7 +189,7 @@ PLAN-ROOT SINK
 |       partitions: 4/4 rows=100
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=32.00MB mem-reservation=0B
+|     mem-estimate=32.00MB mem-reservation=8.00KB
 |     tuple-ids=1 row-size=4B cardinality=10
 |
 00:SCAN HDFS [functional.alltypes t1]
@@ -200,7 +200,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=4B cardinality=7300
 ====
 # Sampling in WITH-clause view.
@@ -208,7 +208,7 @@ with t as (select * from functional.alltypes tablesample system(10) repeatable(1
 select id from t
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -219,6 +219,6 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=4B cardinality=730
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/union.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/union.test b/testdata/workloads/functional-planner/queries/PlannerTest/union.test
index a0d0c26..09ac11b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/union.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/union.test
@@ -3233,7 +3233,7 @@ PLAN-ROOT SINK
 |  tuple-ids=2 row-size=4B cardinality=7300
 |
 F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=128.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB
 00:UNION
 |  pass-through-operands: all
 |  mem-estimate=0B mem-reservation=0B
@@ -3246,7 +3246,7 @@ Per-Host Resources: mem-estimate=128.00MB mem-reservation=0B
 |       partitions: 24/24 rows=7300
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=128.00MB mem-reservation=0B
+|     mem-estimate=128.00MB mem-reservation=32.00KB
 |     tuple-ids=1 row-size=4B cardinality=7300
 |
 01:SCAN HDFS [functional.emptytable, RANDOM]
@@ -3277,7 +3277,7 @@ PLAN-ROOT SINK
 |  tuple-ids=2 row-size=4B cardinality=7300
 |
 F02:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=128.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=128.00MB mem-reservation=32.00KB
 00:UNION
 |  pass-through-operands: all
 |  mem-estimate=0B mem-reservation=0B
@@ -3300,6 +3300,6 @@ Per-Host Resources: mem-estimate=128.00MB mem-reservation=0B
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=4B cardinality=7300
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/admission-reject-min-reservation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/admission-reject-min-reservation.test b/testdata/workloads/functional-query/queries/QueryTest/admission-reject-min-reservation.test
index 119c2fd..de12db1 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/admission-reject-min-reservation.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/admission-reject-min-reservation.test
@@ -5,7 +5,7 @@ select distinct * from functional_parquet.alltypesagg
 ---- CATCH
 minimum memory reservation is greater than memory available to the
  query for buffer reservations. Memory reservation needed given the
- current plan: 68.00 MB. Set mem_limit to at least 143.00 MB.
+ current plan: 68.09 MB. Set mem_limit to at least 100.09 MB.
 ====
 ---- QUERY
 set mem_limit=150mb;
@@ -13,14 +13,14 @@ select distinct * from functional_parquet.alltypesagg
 ---- CATCH
 minimum memory reservation needed is greater than pool max mem resources.
  Pool max mem resources: 10.00 MB.
- Cluster-wide memory reservation needed: 204.00 MB
+ Cluster-wide memory reservation needed: 204.26 MB
 ====
 ---- QUERY
 set buffer_pool_limit=10mb;
 select distinct * from functional_parquet.alltypesagg
 ---- CATCH
 minimum memory reservation is greater than memory available to the query
- for buffer reservations. Increase the buffer_pool_limit to 68.00 MB.
+ for buffer reservations. Increase the buffer_pool_limit to 68.09 MB.
 ====
 ---- QUERY
 set mem_limit=1024;
@@ -29,14 +29,14 @@ from tpch_parquet.lineitem join tpch_parquet.orders on l_orderkey = o_orderkey
 ---- CATCH
 minimum memory reservation is greater than memory available to the
  query for buffer reservations. Memory reservation needed given the
- current plan: 6.75 MB. Set mem_limit to at least 81.75 MB.
+ current plan: 22.75 MB. Set mem_limit to at least 54.75 MB.
 ====
 ---- QUERY
-set mem_limit=80mb;
+set mem_limit=50mb;
 select count(*)
 from tpch_parquet.lineitem join tpch_parquet.orders on l_orderkey = o_orderkey
 ---- CATCH
 minimum memory reservation is greater than memory available to the
  query for buffer reservations. Memory reservation needed given the
- current plan: 18.00 MB. Set mem_limit to at least 93.00 MB.
+ current plan: 34.00 MB. Set mem_limit to at least 66.00 MB.
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
index f8a024a..d725391 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/analytic-fns.test
@@ -1584,9 +1584,10 @@ int, bigint, bigint, double
 ====
 ---- QUERY
 # Check that the a large analytic query can succeed with the minimum buffers (3 buffers
-# for sort, 2 buffer for analytic).
+# for sort, 2 buffer for analytic, plus 32KB extra for the I/O buffers required to scan
+# the < 16KB functional_parquet.alltypes files).
 SET default_spillable_buffer_size=8m;
-SET buffer_pool_limit=40m;
+SET buffer_pool_limit=47m;
 SELECT lag(-180, 13) over (ORDER BY t1.int_col ASC, t2.int_col ASC) AS int_col
 FROM functional_parquet.alltypes t1 CROSS JOIN functional_parquet.alltypes t2 LIMIT 10;
 ---- TYPES

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/codegen-mem-limit.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/codegen-mem-limit.test b/testdata/workloads/functional-query/queries/QueryTest/codegen-mem-limit.test
index dea6066..f7013ae 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/codegen-mem-limit.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/codegen-mem-limit.test
@@ -1,9 +1,8 @@
 =====
 ---- QUERY
+set exec_single_node_rows_threshold=0;
 set mem_limit=100k;
-select *
-from alltypes
-where substr(string_col, 1) = "";
+with t as (values(1), (2), (3), (4)) select min(t1.`1` + t2.`1`) from t t1 join t t2;
 ---- CATCH
 Codegen failed to reserve
 =====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/disk-spill-encryption.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/disk-spill-encryption.test b/testdata/workloads/functional-query/queries/QueryTest/disk-spill-encryption.test
index 2c4fede..48649e5 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/disk-spill-encryption.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/disk-spill-encryption.test
@@ -1,6 +1,6 @@
 ====
 ---- QUERY
-set buffer_pool_limit=10m;
+set buffer_pool_limit=90m;
 set default_spillable_buffer_size=64k;
 select count(*)
 from (select distinct o_orderdate, o_custkey, o_comment

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test
index 92abd5f..53afc32 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level0.test
@@ -5,8 +5,8 @@ explain
 select *
 from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=35.00MB'
-'Per-Host Resource Estimates: Memory=477.41MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=[0-9.]*MB.*
+row_regex:.*Per-Host Resource Estimates: Memory=[0-9.]*MB.*
 ''
 'PLAN-ROOT SINK'
 '04:EXCHANGE [UNPARTITIONED]'

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test
index e65745a..6d58d29 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level1.test
@@ -5,8 +5,8 @@ explain
 select *
 from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=35.00MB'
-'Per-Host Resource Estimates: Memory=477.41MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=[0-9.]*MB.*
+row_regex:.*Per-Host Resource Estimates: Memory=[0-9.]*MB.*
 ''
 'PLAN-ROOT SINK'
 '|'

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
index b1c58f4..17fdd10 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level2.test
@@ -5,8 +5,8 @@ explain
 select *
 from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=35.00MB'
-'Per-Host Resource Estimates: Memory=477.41MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=[0-9.]*MB.*
+row_regex:.*Per-Host Resource Estimates: Memory=[0-9.]*MB.*
 ''
 'F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 '|  Per-Host Resources: mem-estimate=0B mem-reservation=0B'
@@ -18,12 +18,12 @@ from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
 '|  tuple-ids=0,1 row-size=454B cardinality=5757710'
 '|'
 'F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3'
-'Per-Host Resources: mem-estimate=389.41MB mem-reservation=35.00MB runtime-filters-memory=1.00MB'
+row_regex:.*Per-Host Resources: mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB.*
 '02:HASH JOIN [INNER JOIN, BROADCAST]'
 '|  hash predicates: l_orderkey = o_orderkey'
 '|  fk/pk conjuncts: l_orderkey = o_orderkey'
 '|  runtime filters: RF000[bloom] <- o_orderkey'
-'|  mem-estimate=300.41MB mem-reservation=34.00MB spill-buffer=2.00MB'
+row_regex:.*|  mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB spill-buffer=[0-9.]*MB.*
 '|  tuple-ids=0,1 row-size=454B cardinality=5757710'
 '|'
 '|--03:EXCHANGE [BROADCAST]'
@@ -31,14 +31,14 @@ from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
 '|  |  tuple-ids=1 row-size=191B cardinality=1500000'
 '|  |'
 '|  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2'
-'|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B'
+row_regex:.*|  Per-Host Resources: mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB.*
 '|  01:SCAN HDFS [tpch.orders, RANDOM]'
 row_regex:.*partitions=1/1 files=1 size=.*
 '|     stored statistics:'
 row_regex:.*table: rows=1500000 size=.*
 '|       columns: all'
 '|     extrapolated-rows=disabled'
-'|     mem-estimate=88.00MB mem-reservation=0B'
+row_regex:.*|     mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB.*
 '|     tuple-ids=1 row-size=191B cardinality=1500000'
 '|'
 '00:SCAN HDFS [tpch.lineitem, RANDOM]'
@@ -48,7 +48,7 @@ row_regex:.*partitions=1/1 files=1 size=.*
 row_regex:.*table: rows=6001215 size=.*
 '     columns: all'
 '   extrapolated-rows=disabled'
-'   mem-estimate=88.00MB mem-reservation=0B'
+row_regex:.*   mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB.*
 '   tuple-ids=0 row-size=263B cardinality=6001215'
 ====
 ---- QUERY

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
index 625e2b0..4ef3524 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/explain-level3.test
@@ -5,8 +5,8 @@ explain
 select *
 from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=35.00MB'
-'Per-Host Resource Estimates: Memory=477.41MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=[0-9.]*MB.*
+row_regex:.*Per-Host Resource Estimates: Memory=[0-9.]*MB.*
 ''
 'F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
 'Per-Host Resources: mem-estimate=0B mem-reservation=0B'
@@ -18,14 +18,14 @@ from tpch.lineitem join tpch.orders on l_orderkey = o_orderkey;
 '     tuple-ids=0,1 row-size=454B cardinality=5757710'
 ''
 'F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3'
-'Per-Host Resources: mem-estimate=389.41MB mem-reservation=35.00MB runtime-filters-memory=1.00MB'
+row_regex:.*Per-Host Resources: mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB.*
 '  DATASTREAM SINK [FRAGMENT=F02, EXCHANGE=04, UNPARTITIONED]'
 '  |  mem-estimate=0B mem-reservation=0B'
 '  02:HASH JOIN [INNER JOIN, BROADCAST]'
 '  |  hash predicates: l_orderkey = o_orderkey'
 '  |  fk/pk conjuncts: l_orderkey = o_orderkey'
 '  |  runtime filters: RF000[bloom] <- o_orderkey'
-'  |  mem-estimate=300.41MB mem-reservation=34.00MB spill-buffer=2.00MB'
+row_regex:.*  |  mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB spill-buffer=[0-9.]*MB.*
 '  |  tuple-ids=0,1 row-size=454B cardinality=5757710'
 '  |'
 '  |--03:EXCHANGE [BROADCAST]'
@@ -39,11 +39,11 @@ row_regex:.*partitions=1/1 files=1 size=.*
 row_regex:.*table: rows=6001215 size=.*
 '       columns: all'
 '     extrapolated-rows=disabled'
-'     mem-estimate=88.00MB mem-reservation=0B'
+row_regex:.*     mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB.*
 '     tuple-ids=0 row-size=263B cardinality=6001215'
 ''
 'F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2'
-'Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B'
+row_regex:.*Per-Host Resources: mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB.*
 '  DATASTREAM SINK [FRAGMENT=F00, EXCHANGE=03, BROADCAST]'
 '  |  mem-estimate=0B mem-reservation=0B'
 '  01:SCAN HDFS [tpch.orders, RANDOM]'
@@ -52,7 +52,7 @@ row_regex:.*partitions=1/1 files=1 size=.*
 row_regex:.*table: rows=1500000 size=.*
 '       columns: all'
 '     extrapolated-rows=disabled'
-'     mem-estimate=88.00MB mem-reservation=0B'
+row_regex:.*     mem-estimate=[0-9.]*MB mem-reservation=[0-9.]*MB.*
 '     tuple-ids=1 row-size=191B cardinality=1500000'
 ====
 ---- QUERY

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch.test b/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch.test
index c8a80b2..6b213eb 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/nested-types-tpch.test
@@ -162,7 +162,7 @@ BIGINT,BIGINT
 ---- QUERY
 # IMPALA-2376
 # Set memory limit low enough to get the below query to consistently fail.
-# TODO: change query and/or table so we can consistently check for the following error:
+# This was originally a regression test that hit an error like:
 # Failed to allocate buffer for collection '...'.
 set mem_limit=4m;
 select max(cnt) from customer c,
@@ -170,7 +170,7 @@ select max(cnt) from customer c,
 ---- TYPES
 BIGINT
 ---- CATCH
-Memory limit exceeded
+Rejected query from pool default-pool: minimum memory reservation is greater than memory available to the query for buffer reservations.
 ====
 ---- QUERY
 # IMPALA-2473: Scan query with large row size leading to oversized batches.
@@ -237,7 +237,7 @@ bigint, bigint
 # This was originally a regression test for IMPALA-5446: dropped status from
 # Sorter::Reset() when sort cannot get reserved buffer. However with the
 # IMPALA-3200 changes it now succeeds.
-set buffer_pool_limit=100m;
+set buffer_pool_limit=180m;
 select c_custkey, c_name, c_address, c_phone, c_acctbal, c_mktsegment, c_comment,
        o_orderdate, sum(o_totalprice), min(rnum)
 from customer c,

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test b/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test
index 3d92acb..fdca977 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/runtime_row_filters.test
@@ -345,7 +345,7 @@ where l_orderkey = 965 and l_extendedprice * l_tax = p_retailprice;
 # consumption / spilling behaviour.
 ####################################################
 
-SET BUFFER_POOL_LIMIT=40m;
+SET BUFFER_POOL_LIMIT=80m;
 SET RUNTIME_FILTER_MODE=GLOBAL;
 SET RUNTIME_FILTER_WAIT_TIME_MS=$RUNTIME_FILTER_WAIT_TIME_MS;
 SET RUNTIME_BLOOM_FILTER_SIZE=16M;
@@ -396,15 +396,14 @@ SET RUNTIME_FILTER_WAIT_TIME_MS=$RUNTIME_FILTER_WAIT_TIME_MS;
 SET RUNTIME_FILTER_MIN_SIZE=128MB;
 SET RUNTIME_FILTER_MAX_SIZE=500MB;
 # Query would have been admitted if memory for runtime filters was not accounted for.
-SET BUFFER_POOL_LIMIT=140MB;
+SET BUFFER_POOL_LIMIT=290MB;
 select STRAIGHT_JOIN * from alltypes a join [SHUFFLE] alltypes b
     on a.month = b.id and b.int_col = -3
 ---- RESULTS
 ---- CATCH
-ImpalaBeeswaxException: INNER EXCEPTION: <class 'beeswaxd.ttypes.BeeswaxException'>
- MESSAGE: Rejected query from pool default-pool: minimum memory reservation is
+Rejected query from pool default-pool: minimum memory reservation is
  greater than memory available to the query for buffer reservations. Increase
- the buffer_pool_limit to 290.00 MB. See the query profile for more information
+ the buffer_pool_limit to 290.17 MB. See the query profile for more information
  about the per-node memory requirements.
 ====
 ---- QUERY
@@ -414,7 +413,7 @@ SET RUNTIME_FILTER_WAIT_TIME_MS=$RUNTIME_FILTER_WAIT_TIME_MS;
 SET RUNTIME_FILTER_MIN_SIZE=128MB;
 SET RUNTIME_FILTER_MAX_SIZE=500MB;
 # This would run perfectly with just enough memory provided by the buffer pool.
-SET BUFFER_POOL_LIMIT=290MB;
+SET BUFFER_POOL_LIMIT=295MB;
 select STRAIGHT_JOIN * from alltypes a join [SHUFFLE] alltypes b
     on a.month = b.id and b.int_col = -3
 ---- RESULTS

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/scanners.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/scanners.test b/testdata/workloads/functional-query/queries/QueryTest/scanners.test
index bd5f496..a000a15 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/scanners.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/scanners.test
@@ -110,3 +110,10 @@ on (v.x = t3.bool_col) where t3.bool_col = true
 ---- TYPES
 BIGINT
 ====
+---- QUERY
+# Scan an empty unpartitioned table.
+select * from emptytable;
+---- RESULTS
+---- TYPES
+STRING,INT
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/set.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/set.test b/testdata/workloads/functional-query/queries/QueryTest/set.test
index 32ad938..7983224 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/set.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/set.test
@@ -129,7 +129,7 @@ select 1
 set mem_limit=1;
 select count(string_col) from functional.alltypestiny
 ---- CATCH
-Memory limit exceeded
+Rejected query from pool default-pool: minimum memory reservation
 ====
 ---- QUERY
 # Set mem_limit back to unlimited and query should succeed again.

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/spilling-aggs.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-aggs.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-aggs.test
index 05552d2..20813c7 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling-aggs.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-aggs.test
@@ -1,6 +1,6 @@
 ====
 ---- QUERY
-set buffer_pool_limit=10m;
+set buffer_pool_limit=34m;
 select l_orderkey, count(*)
 from lineitem
 group by 1
@@ -25,7 +25,7 @@ row_regex: .*RowsPassedThrough: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
 # Test query with string grouping column and string agg columns
-set buffer_pool_limit=10m;
+set buffer_pool_limit=82m;
 set num_nodes=1;
 select l_returnflag, l_orderkey, avg(l_tax), min(l_shipmode)
 from lineitem
@@ -39,10 +39,9 @@ order by 1,2 limit 3
 STRING, BIGINT, DECIMAL, STRING
 ---- RUNTIME_PROFILE
 row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
-row_regex: .*NumRepartitions: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
-set buffer_pool_limit=10m;
+set buffer_pool_limit=34m;
 select l_orderkey, count(*)
 from lineitem
 group by 1
@@ -66,7 +65,7 @@ row_regex: .*RowsPassedThrough: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
 # Test query with string grouping column
-set buffer_pool_limit=10m;
+set buffer_pool_limit=34m;
 set num_nodes=1;
 select l_comment, count(*)
 from lineitem
@@ -82,11 +81,10 @@ order by count(*) desc limit 5
 STRING, BIGINT
 ---- RUNTIME_PROFILE
 row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
-row_regex: .*NumRepartitions: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
 # Test query with string grouping column and string agg columns
-set buffer_pool_limit=10m;
+set buffer_pool_limit=82m;
 set num_nodes=1;
 select l_returnflag, l_orderkey, round(avg(l_tax),2), min(l_shipmode)
 from lineitem
@@ -101,11 +99,10 @@ STRING, BIGINT, DECIMAL, STRING
 ---- RUNTIME_PROFILE
 # Verify that spilling happened in the agg.
 row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
-row_regex: .*NumRepartitions: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
 # Test with non-scalar intermediate state (avg() uses fixed intermediate value).
-set buffer_pool_limit=10m;
+set buffer_pool_limit=34m;
 select l_orderkey, avg(l_orderkey)
 from lineitem
 group by 1
@@ -125,7 +122,7 @@ row_regex: .*RowsPassedThrough: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
 # Test aggregation spill with group_concat distinct
-set buffer_pool_limit=30m;
+set buffer_pool_limit=78m;
 select l_orderkey, count(*), group_concat(distinct l_linestatus, '|')
 from lineitem
 group by 1
@@ -171,7 +168,7 @@ row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
 # Merge aggregation requires 17 buffers and preaggregation requires 16 buffers
 # plus 1mb of hash tables. The buffer size is 256k for this test.
 set max_row_size=256k;
-set buffer_pool_limit=9472k;
+set buffer_pool_limit="57.25M";
 select count(*)
 from (select distinct * from orders) t
 ---- TYPES

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj-no-deny-reservation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj-no-deny-reservation.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj-no-deny-reservation.test
index 420e721..7678675 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj-no-deny-reservation.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj-no-deny-reservation.test
@@ -13,7 +13,7 @@
 # except:
 # * even-numbered l_orderkey values, because there is a NULL on the build side.
 # * l_orderkey values > 5500000, because there is a NULL on the probe side
-set buffer_pool_limit=12m;
+set buffer_pool_limit=110m;
 select l_orderkey, l_partkey, l_suppkey, l_linenumber
 from lineitem
 where l_suppkey = 4162 and l_shipmode = 'AIR' and l_returnflag = 'A' and
@@ -54,8 +54,9 @@ order by 1,2,3,4
 BIGINT,BIGINT,BIGINT,INT
 ====
 ---- QUERY
-set buffer_pool_limit=7m;
-# Execute NAAJ query 1 without enough memory to fit the null build rows in memory.
+set debug_action="-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0";
+# Execute NAAJ query 1 with a debug action so that it can not get enough memory to fit
+# the null build rows in memory.
 select l_orderkey, l_partkey, l_suppkey, l_linenumber
 from lineitem
 where l_suppkey = 4162 and l_shipmode = 'AIR' and l_returnflag = 'A' and

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj.test
index 4c0d790..39ae2bb 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-naaj.test
@@ -21,7 +21,7 @@
 #   where l_orderkey % 2 = 1 and p_retailprice = l_extendedprice * l_tax
 #   order by 1,2,3,4 limit 5
 #
-set buffer_pool_limit=10m;
+set buffer_pool_limit=90m;
 select l_orderkey, l_partkey, l_suppkey, l_linenumber
 from lineitem
 where if(l_orderkey % 2 = 0, NULL, l_partkey) not in (
@@ -41,7 +41,7 @@ BIGINT,BIGINT,BIGINT,INT
 ---- QUERY
 # Execute NAAJ query 2 in-memory only without enough memory to complete.
 set scratch_limit=0;
-set buffer_pool_limit=10m;
+set buffer_pool_limit=90m;
 select l_orderkey, l_partkey, l_suppkey, l_linenumber
 from lineitem
 where if(l_orderkey % 2 = 0, NULL, l_partkey) not in (
@@ -65,7 +65,7 @@ Could not free memory by spilling to disk: scratch_limit is 0
 #   where p_partkey = 2 and l_partkey % 2 = 1
 #   order by 1,2,3,4
 #
-set buffer_pool_limit=10m;
+set buffer_pool_limit=90m;
 select l_orderkey, l_partkey, l_suppkey, l_linenumber
 from lineitem
 where l_partkey not in (
@@ -86,7 +86,7 @@ BIGINT,BIGINT,BIGINT,INT
 # =======================================================================================
 # This returns one less row than the previous query because l_partkey=116009 is
 # replaced with a NULL.
-set buffer_pool_limit=10m;
+set buffer_pool_limit=90m;
 select l_orderkey, l_partkey, l_suppkey, l_linenumber
 from lineitem
 where if(l_partkey % 5 != 1, NULL, l_partkey) not in (

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test
new file mode 100644
index 0000000..f89e537
--- /dev/null
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-no-debug-action.test
@@ -0,0 +1,66 @@
+# This file contains tests where we don't want the python test framework to supply the
+# debug_action value because the test won't succeed with all possible debug_action values.
+====
+---- QUERY
+# Tests for the case where a spilled partition has 0 probe rows and so we don't build the
+# hash table in a partitioned hash join. Always runs with the minimum reservation to force
+# spilling.
+# INNER JOIN
+set debug_action="-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0";
+select straight_join count(*)
+from
+lineitem a, lineitem b
+where
+a.l_partkey = 1 and
+a.l_orderkey = b.l_orderkey;
+---- TYPES
+BIGINT
+---- RESULTS
+173
+---- RUNTIME_PROFILE
+row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
+====
+---- QUERY
+# spilled partition with 0 probe rows, NULL AWARE LEFT ANTI JOIN
+set debug_action="-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0";
+select straight_join count(*)
+from
+lineitem a
+where
+a.l_partkey not in (select l_partkey from lineitem where l_partkey > 10)
+and a.l_partkey < 1000;
+---- TYPES
+BIGINT
+---- RESULTS
+287
+---- RUNTIME_PROFILE
+row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
+====
+---- QUERY
+# spilled partition with 0 probe rows, RIGHT OUTER JOIN
+set debug_action="-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0";
+select straight_join count(*)
+from
+supplier right outer join lineitem on s_suppkey = l_suppkey
+where s_acctbal > 0 and s_acctbal < 10;
+---- TYPES
+BIGINT
+---- RESULTS
+12138
+---- RUNTIME_PROFILE
+row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
+====
+---- QUERY
+# spilled partition with 0 probe rows, RIGHT ANTI JOIN
+set debug_action="-1:OPEN:SET_DENY_RESERVATION_PROBABILITY@1.0";
+with x as (select * from supplier limit 10)
+select straight_join count(*)
+from
+x right anti join lineitem on s_suppkey + 100 = l_suppkey;
+---- TYPES
+BIGINT
+---- RESULTS
+5995258
+---- RUNTIME_PROFILE
+row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/spilling-sorts-exhaustive.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling-sorts-exhaustive.test b/testdata/workloads/functional-query/queries/QueryTest/spilling-sorts-exhaustive.test
index 415fe5d..7d65523 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling-sorts-exhaustive.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling-sorts-exhaustive.test
@@ -3,7 +3,7 @@
 # Test sort with small char column materialized by exprs.
 # Set low memory limit to force spilling.
 # IMPALA-3332: comparator makes local allocations that cause runaway memory consumption.
-set buffer_pool_limit=4m;
+set buffer_pool_limit=28m;
 set mem_limit=200m;
 set disable_outermost_topn=1;
 select cast(l_comment as char(50))
@@ -41,7 +41,7 @@ row_regex: .*TotalMergesPerformed: .* \([1-9][0-9]*\)
 ---- QUERY
 # Test sort with small input char column materialized before sort.
 set mem_limit=200m;
-set buffer_pool_limit=4m;
+set buffer_pool_limit=28m;
 set disable_outermost_topn=1;
 select char_col
 from (select cast(l_comment as char(50)) char_col
@@ -80,7 +80,7 @@ row_regex: .*TotalMergesPerformed: .* \([1-9][0-9]*\)
 # Test sort with large input char column materialized before sort.
 # Set low memory limit to force spilling.
 set mem_limit=200m;
-set buffer_pool_limit=4m;
+set buffer_pool_limit=28m;
 set disable_outermost_topn=1;
 select char_col
 from (select cast(l_comment as char(200)) char_col
@@ -118,7 +118,7 @@ row_regex: .*TotalMergesPerformed: .* \([1-9][0-9]*\)
 ---- QUERY
 # Test sort with varchar column materialized by exprs.
 # Set low memory limit to force spilling.
-set buffer_pool_limit=4m;
+set buffer_pool_limit=28m;
 # IMPALA-3332: comparator makes local allocations that cause runaway memory consumption.
 set mem_limit=200m;
 set disable_outermost_topn=1;
@@ -158,7 +158,7 @@ row_regex: .*TotalMergesPerformed: .* \([1-9][0-9]*\)
 # Test sort with input varchar column materialized before sort.
 # Set low memory limit to force spilling.
 set mem_limit=200m;
-set buffer_pool_limit=4m;
+set buffer_pool_limit=28m;
 set disable_outermost_topn=1;
 select char_col
 from (select cast(l_comment as varchar(50)) char_col

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/spilling.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/spilling.test b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
index 6639d82..23d9271 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/spilling.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/spilling.test
@@ -1,6 +1,6 @@
 ====
 ---- QUERY
-set buffer_pool_limit=15m;
+set buffer_pool_limit=215m;
 select count(l1.l_tax)
 from
 lineitem l1,
@@ -22,7 +22,7 @@ BIGINT
 row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
-set buffer_pool_limit=2m;
+set buffer_pool_limit=50m;
 select max(t1.total_count), max(t1.l_shipinstruct), max(t1.l_comment) from
 (select l_shipinstruct, l_comment, count(*) over () total_count from lineitem) t1
 ---- RESULTS
@@ -60,7 +60,7 @@ row_regex: .*SpilledPartitions: 0 .*
 # Adding TPCH-Q21 in the spilling test to check for IMPALA-1471 (spilling left anti
 # and left outer joins were returning wrong results).
 # Q21 - Suppliers Who Kept Orders Waiting Query
-set buffer_pool_limit=26m;
+set buffer_pool_limit=200m;
 select
   s_name,
   count(*) as numwait
@@ -211,7 +211,7 @@ row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
 ---- QUERY
 # IMPALA-1346/IMPALA-1546: fix sorter memory management so that it can complete
 # successfully when in same pipeline as a spilling join.
-set buffer_pool_limit=50m;
+set buffer_pool_limit=170m;
 set disable_outermost_topn=1;
 select * from lineitem
   inner join orders on l_orderkey = o_orderkey
@@ -246,68 +246,6 @@ row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
 row_regex: .*TotalMergesPerformed: .* \([1-9][0-9]*\)
 ====
 ---- QUERY
-# Tests for the case where a spilled partition has 0 probe rows and so we don't build the
-# hash table in a partitioned hash join.
-# INNER JOIN
-set buffer_pool_limit=10m;
-select straight_join count(*)
-from
-lineitem a, lineitem b
-where
-a.l_partkey = 1 and
-a.l_orderkey = b.l_orderkey;
----- TYPES
-BIGINT
----- RESULTS
-173
----- RUNTIME_PROFILE
-row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
-====
----- QUERY
-# spilled partition with 0 probe rows, NULL AWARE LEFT ANTI JOIN
-set buffer_pool_limit=10m;
-select straight_join count(*)
-from
-lineitem a
-where
-a.l_partkey not in (select l_partkey from lineitem where l_partkey > 10)
-and a.l_partkey < 1000;
----- TYPES
-BIGINT
----- RESULTS
-287
----- RUNTIME_PROFILE
-row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
-====
----- QUERY
-# spilled partition with 0 probe rows, RIGHT OUTER JOIN
-set buffer_pool_limit=10m;
-select straight_join count(*)
-from
-supplier right outer join lineitem on s_suppkey = l_suppkey
-where s_acctbal > 0 and s_acctbal < 10;
----- TYPES
-BIGINT
----- RESULTS
-12138
----- RUNTIME_PROFILE
-row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
-====
----- QUERY
-# spilled partition with 0 probe rows, RIGHT ANTI JOIN
-set buffer_pool_limit=30m;
-with x as (select * from supplier limit 10)
-select straight_join count(*)
-from
-x right anti join lineitem on s_suppkey + 100 = l_suppkey;
----- TYPES
-BIGINT
----- RESULTS
-5995258
----- RUNTIME_PROFILE
-row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
-====
----- QUERY
 # IMPALA-5173: spilling hash join feeding into right side of nested loop join.
 # Equivalent to:
 #   select *
@@ -315,7 +253,7 @@ row_regex: .*NumHashTableBuildsSkipped: .* \([1-9][0-9]*\)
 #   where l1.l_quantity = 31.0 and l1.l_tax = 0.03 and l1.l_orderkey <= 100000
 # order by l_orderkey, l_partkey, l_suppkey, l_linenumber
 # limit 5
-set buffer_pool_limit=9m;
+set buffer_pool_limit=177m;
 set num_nodes=1;
 select straight_join l.*
 from
@@ -345,7 +283,7 @@ bigint,bigint,bigint,int,decimal,decimal,decimal,decimal,string,string,string,st
 ====
 ---- QUERY
 # Test spilling aggregation when grouping by nondeterministic expression
-set buffer_pool_limit=5m;
+set buffer_pool_limit=79m;
 set num_nodes=1;
 select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_comment
 from tpch_parquet.lineitem
@@ -359,7 +297,7 @@ row_regex: .*SpilledPartitions: .* \([1-9][0-9]*\)
 ---- QUERY
 # Test spilling join with many duplicates in join key. We don't expect this to succeed
 # with a memory constraint: see IMPALA-4857.
-set buffer_pool_limit=20m;
+set buffer_pool_limit=167m;
 select *
 from lineitem l1 join lineitem l2 on l1.l_linenumber = l2.l_linenumber
 ---- CATCH

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
index 7da7baf..836ed0a 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/stats-extrapolation.test
@@ -15,7 +15,6 @@ explain select id from alltypes;
 '     partitions: 0/12 rows=unavailable'
 '     columns: unavailable'
 '   extrapolated-rows=unavailable'
-'   mem-estimate=16.00MB mem-reservation=0B'
 '   tuple-ids=0 row-size=4B cardinality=unavailable'
 ---- TYPES
 STRING
@@ -53,14 +52,14 @@ STRING,STRING,BIGINT,BIGINT,BIGINT,STRING,STRING,STRING,STRING,STRING,STRING
 # Stats are available now.
 explain select id from alltypes;
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=0B'
-'Per-Host Resource Estimates: Memory=16.00MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=.*
+row_regex:.*Per-Host Resource Estimates: Memory=.*
 'Codegen disabled by planner'
 ''
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
-'|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*Per-Host Resources: mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
-'|  mem-estimate=0B mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN HDFS [$DATABASE.alltypes]'
 row_regex:.*partitions=12/12 files=12 size=.*
@@ -69,7 +68,7 @@ row_regex:.*table: rows=3650 size=.*
 '     partitions: 0/12 rows=unavailable'
 '     columns: all'
 '   extrapolated-rows=3650'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=3650'
 ---- TYPES
 STRING
@@ -78,14 +77,14 @@ STRING
 # Select a subset of partitions.
 explain select id from alltypes where month in (1, 2, 3);
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=0B'
-'Per-Host Resource Estimates: Memory=16.00MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=.*
+row_regex:.*Per-Host Resource Estimates: Memory=.*
 'Codegen disabled by planner'
 ''
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
-'|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
-'|  mem-estimate=0B mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN HDFS [$DATABASE.alltypes]'
 row_regex:.*partitions=3/12 files=3 size=.*
@@ -94,7 +93,7 @@ row_regex:.*table: rows=3650 size=.*
 '     partitions: 0/3 rows=unavailable'
 '     columns: all'
 '   extrapolated-rows=904'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=904'
 ---- TYPES
 STRING
@@ -105,13 +104,13 @@ insert into alltypes partition(year, month)
 select * from functional_parquet.alltypes where year = 2009;
 explain select id from alltypes;
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=0B'
-'Per-Host Resource Estimates: Memory=16.00MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=.*
+row_regex:.*Per-Host Resource Estimates: Memory=.*
 ''
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
-'|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*Per-Host Resources: mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
-'|  mem-estimate=0B mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN HDFS [$DATABASE.alltypes]'
 row_regex:.*partitions=12/12 files=24 size=.*
@@ -120,7 +119,7 @@ row_regex:.*table: rows=3650 size=.*
 '     partitions: 0/12 rows=unavailable'
 '     columns: all'
 '   extrapolated-rows=7300'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=7300'
 ---- TYPES
 STRING
@@ -131,14 +130,14 @@ insert into alltypes partition(year, month)
 select * from functional_parquet.alltypes where year = 2010;
 explain select id from alltypes where year = 2010;
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=0B'
-'Per-Host Resource Estimates: Memory=16.00MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=.*
+row_regex:.*Per-Host Resource Estimates: Memory=16.00MB'
 'Codegen disabled by planner'
 ''
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
-'|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*Per-Host Resources: mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
-'|  mem-estimate=0B mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN HDFS [$DATABASE.alltypes]'
 row_regex:.*partitions=12/24 files=12 size=.*
@@ -147,7 +146,7 @@ row_regex:.*table: rows=3650 size=.*
 '     partitions: 0/12 rows=unavailable'
 '     columns: all'
 '   extrapolated-rows=3651'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=3651'
 ---- TYPES
 STRING
@@ -157,14 +156,14 @@ STRING
 compute stats alltypes;
 explain select id from alltypes where year = 2010;
 ---- RESULTS: VERIFY_IS_EQUAL
-'Max Per-Host Resource Reservation: Memory=0B'
-'Per-Host Resource Estimates: Memory=16.00MB'
+row_regex:.*Max Per-Host Resource Reservation: Memory=.*
+row_regex:.*Per-Host Resource Estimates: Memory=16.00MB'
 'Codegen disabled by planner'
 ''
 'F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1'
-'|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*Per-Host Resources: mem-estimate=.* mem-reservation=.*
 'PLAN-ROOT SINK'
-'|  mem-estimate=0B mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '|'
 '00:SCAN HDFS [$DATABASE.alltypes]'
 row_regex:.*partitions=12/24 files=12 size=.*
@@ -173,7 +172,7 @@ row_regex:.*table: rows=10950 size=.*
 '     partitions: 0/12 rows=unavailable'
 '     columns: all'
 '   extrapolated-rows=3651'
-'   mem-estimate=16.00MB mem-reservation=0B'
+row_regex:.*mem-estimate=.* mem-reservation=.*
 '   tuple-ids=0 row-size=4B cardinality=3651'
 ---- TYPES
 STRING
@@ -188,7 +187,6 @@ explain select id from alltypes;
 '     partitions: 0/24 rows=unavailable'
 '     columns: unavailable'
 '   extrapolated-rows=unavailable'
-'   mem-estimate=16.00MB mem-reservation=0B'
 '   tuple-ids=0 row-size=4B cardinality=unavailable'
 ---- TYPES
 STRING


[08/15] impala git commit: IMPALA-4835: Part 1: simplify I/O mgr mem mgmt and cancellation

Posted by ta...@apache.org.
IMPALA-4835: Part 1: simplify I/O mgr mem mgmt and cancellation

In preparation for switching the I/O mgr to the buffer pool, this
removes and cleans up a lot of code so that the switchover patch starts
from a cleaner slate.

* Remove the free buffer cache (which will be replaced by buffer pool's
  own caching).
* Make memory limit exceeded error checking synchronous (in anticipation
  of having to propagate buffer pool errors synchronously).
* Simplify error propagation - remove the (ineffectual) code that
  enqueued BufferDescriptors containing error statuses.
* Document locking scheme better in a few places, make it part of the
  function signature when it seemed reasonable.
* Move ReturnBuffer() to ScanRange, because it is intrinsically
  connected with the lifecycle of a scan range.
* Separate external ReturnBuffer() and internal CleanUpBuffer()
  interfaces - previously callers of ReturnBuffer() were fudging
  the num_buffers_in_reader accounting to make the external interface work.
* Eliminate redundant state in ScanRange: 'eosr_returned_' and
  'is_cancelled_'.
* Clarify the logic around calling Close() for the last
  BufferDescriptor.
  -> There appeared to be an implicit assumption that buffers would be
     freed in the order they were returned from the scan range, so that
     the "eos" buffer was returned last. Instead just count the number
     of outstanding buffers to detect the last one.
  -> Touching the is_cancelled_ field without holding a lock was hard to
     reason about - violated locking rules and it was unclear that it
     was race-free.
* Remove DiskIoMgr::Read() to simplify the interface. It is trivial to
  inline at the callsites.

This will probably regress performance somewhat because of the cache
removal, so my plan is to merge it around the same time as switching
the I/O mgr to allocate from the buffer pool. I'm keeping the patches
separate to make reviewing easier.

Testing:
* Ran exhaustive tests
* Ran the disk-io-mgr-stress-test overnight

Change-Id: If5cb42437d11c13bc4a55c3ab426b66777332bd1
Reviewed-on: http://gerrit.cloudera.org:8080/8414
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Tim Armstrong <ta...@cloudera.com>
Reviewed-on: http://gerrit.cloudera.org:8080/9415


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/3b3bf871
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/3b3bf871
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/3b3bf871

Branch: refs/heads/2.x
Commit: 3b3bf8710d88458c4204fd4151aa79239ffe4488
Parents: 7c4689c
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Sun Oct 29 12:38:47 2017 -0700
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Fri Feb 23 22:51:01 2018 +0000

----------------------------------------------------------------------
 be/src/exec/hdfs-parquet-scanner.cc     |   5 +-
 be/src/exec/hdfs-scan-node-base.cc      |  31 +-
 be/src/exec/hdfs-scan-node.cc           |   5 +-
 be/src/exec/scanner-context.cc          |   9 +-
 be/src/runtime/exec-env.cc              |   7 +-
 be/src/runtime/io/disk-io-mgr-stress.cc |  20 +-
 be/src/runtime/io/disk-io-mgr-test.cc   | 171 +++------
 be/src/runtime/io/disk-io-mgr.cc        | 504 +++++----------------------
 be/src/runtime/io/disk-io-mgr.h         | 124 +------
 be/src/runtime/io/request-context.cc    | 148 ++++++--
 be/src/runtime/io/request-context.h     | 187 ++++++----
 be/src/runtime/io/request-ranges.h      | 107 +++---
 be/src/runtime/io/scan-range.cc         | 142 ++++----
 be/src/runtime/mem-tracker.h            |   1 -
 be/src/runtime/test-env.cc              |   2 +-
 be/src/runtime/tmp-file-mgr-test.cc     |   3 +-
 be/src/runtime/tmp-file-mgr.cc          |   2 +-
 be/src/util/impalad-metrics.cc          |  13 +-
 be/src/util/impalad-metrics.h           |   9 -
 19 files changed, 579 insertions(+), 911 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/exec/hdfs-parquet-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-parquet-scanner.cc b/be/src/exec/hdfs-parquet-scanner.cc
index e279369..bb3d091 100644
--- a/be/src/exec/hdfs-parquet-scanner.cc
+++ b/be/src/exec/hdfs-parquet-scanner.cc
@@ -1440,11 +1440,12 @@ Status HdfsParquetScanner::ProcessFooter() {
 
     unique_ptr<BufferDescriptor> io_buffer;
     RETURN_IF_ERROR(
-        io_mgr->Read(scan_node_->reader_context(), metadata_range, &io_buffer));
+        io_mgr->AddScanRange(scan_node_->reader_context(), metadata_range, true));
+    RETURN_IF_ERROR(metadata_range->GetNext(&io_buffer));
     DCHECK_EQ(io_buffer->buffer(), metadata_buffer.buffer());
     DCHECK_EQ(io_buffer->len(), metadata_size);
     DCHECK(io_buffer->eosr());
-    io_mgr->ReturnBuffer(move(io_buffer));
+    metadata_range->ReturnBuffer(move(io_buffer));
   }
 
   // Deserialize file header

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/exec/hdfs-scan-node-base.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node-base.cc b/be/src/exec/hdfs-scan-node-base.cc
index 2f3d0c5..0660b9b 100644
--- a/be/src/exec/hdfs-scan-node-base.cc
+++ b/be/src/exec/hdfs-scan-node-base.cc
@@ -352,13 +352,10 @@ Status HdfsScanNodeBase::Open(RuntimeState* state) {
   num_scanner_threads_started_counter_ =
       ADD_COUNTER(runtime_profile(), NUM_SCANNER_THREADS_STARTED, TUnit::UNIT);
 
-  runtime_state_->io_mgr()->set_bytes_read_counter(
-      reader_context_.get(), bytes_read_counter());
-  runtime_state_->io_mgr()->set_read_timer(reader_context_.get(), read_timer());
-  runtime_state_->io_mgr()->set_active_read_thread_counter(
-      reader_context_.get(), &active_hdfs_read_thread_counter_);
-  runtime_state_->io_mgr()->set_disks_access_bitmap(
-      reader_context_.get(), &disks_accessed_bitmap_);
+  reader_context_->set_bytes_read_counter(bytes_read_counter());
+  reader_context_->set_read_timer(read_timer());
+  reader_context_->set_active_read_thread_counter(&active_hdfs_read_thread_counter_);
+  reader_context_->set_disks_accessed_bitmap(&disks_accessed_bitmap_);
 
   average_scanner_thread_concurrency_ = runtime_profile()->AddSamplingCounter(
       AVERAGE_SCANNER_THREAD_CONCURRENCY, &active_scanner_thread_counter_);
@@ -824,20 +821,14 @@ void HdfsScanNodeBase::StopAndFinalizeCounters() {
       Substitute("Codegen enabled: $0 out of $1", num_enabled, total));
 
   if (reader_context_ != nullptr) {
-    bytes_read_local_->Set(
-        runtime_state_->io_mgr()->bytes_read_local(reader_context_.get()));
-    bytes_read_short_circuit_->Set(
-        runtime_state_->io_mgr()->bytes_read_short_circuit(reader_context_.get()));
-    bytes_read_dn_cache_->Set(
-        runtime_state_->io_mgr()->bytes_read_dn_cache(reader_context_.get()));
-    num_remote_ranges_->Set(static_cast<int64_t>(
-        runtime_state_->io_mgr()->num_remote_ranges(reader_context_.get())));
-    unexpected_remote_bytes_->Set(
-        runtime_state_->io_mgr()->unexpected_remote_bytes(reader_context_.get()));
-    cached_file_handles_hit_count_->Set(
-        runtime_state_->io_mgr()->cached_file_handles_hit_count(reader_context_.get()));
+    bytes_read_local_->Set(reader_context_->bytes_read_local());
+    bytes_read_short_circuit_->Set(reader_context_->bytes_read_short_circuit());
+    bytes_read_dn_cache_->Set(reader_context_->bytes_read_dn_cache());
+    num_remote_ranges_->Set(reader_context_->num_remote_ranges());
+    unexpected_remote_bytes_->Set(reader_context_->unexpected_remote_bytes());
+    cached_file_handles_hit_count_->Set(reader_context_->cached_file_handles_hit_count());
     cached_file_handles_miss_count_->Set(
-        runtime_state_->io_mgr()->cached_file_handles_miss_count(reader_context_.get()));
+        reader_context_->cached_file_handles_miss_count());
 
     if (unexpected_remote_bytes_->value() >= UNEXPECTED_REMOTE_BYTES_WARN_THRESHOLD) {
       runtime_state_->LogError(ErrorMsg(TErrorCode::GENERAL, Substitute(

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/exec/hdfs-scan-node.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/hdfs-scan-node.cc b/be/src/exec/hdfs-scan-node.cc
index 710a8af..b32a743 100644
--- a/be/src/exec/hdfs-scan-node.cc
+++ b/be/src/exec/hdfs-scan-node.cc
@@ -26,6 +26,7 @@
 #include "exec/scanner-context.h"
 #include "runtime/descriptors.h"
 #include "runtime/fragment-instance-state.h"
+#include "runtime/io/request-context.h"
 #include "runtime/runtime-filter.inline.h"
 #include "runtime/runtime-state.h"
 #include "runtime/mem-tracker.h"
@@ -549,9 +550,7 @@ Status HdfsScanNode::ProcessSplit(const vector<FilterContext>& filter_ctxs,
 void HdfsScanNode::SetDoneInternal() {
   if (done_) return;
   done_ = true;
-  if (reader_context_ != nullptr) {
-    runtime_state_->io_mgr()->CancelContext(reader_context_.get());
-  }
+  if (reader_context_ != nullptr) reader_context_->Cancel();
   materialized_row_batches_->Shutdown();
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/exec/scanner-context.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/scanner-context.cc b/be/src/exec/scanner-context.cc
index abdde07..79e7a85 100644
--- a/be/src/exec/scanner-context.cc
+++ b/be/src/exec/scanner-context.cc
@@ -143,8 +143,10 @@ Status ScannerContext::Stream::GetNextBuffer(int64_t read_past_size) {
     ScanRange* range = parent_->scan_node_->AllocateScanRange(
         scan_range_->fs(), filename(), read_past_buffer_size, offset, partition_id,
         scan_range_->disk_id(), false, BufferOpts::Uncached());
-    RETURN_IF_ERROR(parent_->state_->io_mgr()->Read(
-        parent_->scan_node_->reader_context(), range, &io_buffer_));
+    RETURN_IF_ERROR(parent_->state_->io_mgr()->AddScanRange(
+        parent_->scan_node_->reader_context(), range, true));
+    RETURN_IF_ERROR(range->GetNext(&io_buffer_));
+    DCHECK(io_buffer_->eosr());
   }
 
   DCHECK(io_buffer_ != nullptr);
@@ -324,7 +326,8 @@ Status ScannerContext::Stream::CopyIoToBoundary(int64_t num_bytes) {
 
 void ScannerContext::Stream::ReturnIoBuffer() {
   DCHECK(io_buffer_ != nullptr);
-  ExecEnv::GetInstance()->disk_io_mgr()->ReturnBuffer(move(io_buffer_));
+  ScanRange* range = io_buffer_->scan_range();
+  range->ReturnBuffer(move(io_buffer_));
   io_buffer_pos_ = nullptr;
   io_buffer_bytes_left_ = 0;
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/exec-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.cc b/be/src/runtime/exec-env.cc
index 51eece5..2f19786 100644
--- a/be/src/runtime/exec-env.cc
+++ b/be/src/runtime/exec-env.cc
@@ -36,9 +36,9 @@
 #include "runtime/client-cache.h"
 #include "runtime/coordinator.h"
 #include "runtime/data-stream-mgr.h"
-#include "runtime/io/disk-io-mgr.h"
 #include "runtime/hbase-table-factory.h"
 #include "runtime/hdfs-fs-cache.h"
+#include "runtime/io/disk-io-mgr.h"
 #include "runtime/krpc-data-stream-mgr.h"
 #include "runtime/lib-cache.h"
 #include "runtime/mem-tracker.h"
@@ -367,10 +367,7 @@ Status ExecEnv::Init() {
   LOG(INFO) << "Buffer pool limit: "
             << PrettyPrinter::Print(buffer_pool_limit, TUnit::BYTES);
 
-  RETURN_IF_ERROR(disk_io_mgr_->Init(mem_tracker_.get()));
-
-  mem_tracker_->AddGcFunction(
-      [this](int64_t bytes_to_free) { disk_io_mgr_->GcIoBuffers(bytes_to_free); });
+  RETURN_IF_ERROR(disk_io_mgr_->Init());
 
   // Start services in order to ensure that dependencies between them are met
   if (enable_webserver_) {

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/io/disk-io-mgr-stress.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-stress.cc b/be/src/runtime/io/disk-io-mgr-stress.cc
index 8815357..cfe71ab 100644
--- a/be/src/runtime/io/disk-io-mgr-stress.cc
+++ b/be/src/runtime/io/disk-io-mgr-stress.cc
@@ -77,7 +77,7 @@ DiskIoMgrStress::DiskIoMgrStress(int num_disks, int num_threads_per_disk,
 
   io_mgr_.reset(new DiskIoMgr(num_disks, num_threads_per_disk, num_threads_per_disk,
       MIN_READ_BUFFER_SIZE, MAX_READ_BUFFER_SIZE));
-  Status status = io_mgr_->Init(&mem_tracker_);
+  Status status = io_mgr_->Init();
   CHECK(status.ok());
 
   // Initialize some data files.  It doesn't really matter how many there are.
@@ -137,7 +137,7 @@ void DiskIoMgrStress::ClientThread(int client_id) {
 
         // Copy the bytes from this read into the result buffer.
         memcpy(read_buffer + file_offset, buffer->buffer(), buffer->len());
-        io_mgr_->ReturnBuffer(move(buffer));
+        range->ReturnBuffer(move(buffer));
         bytes_read += len;
 
         CHECK_GE(bytes_read, 0);
@@ -159,6 +159,7 @@ void DiskIoMgrStress::ClientThread(int client_id) {
     // Unregister the old client and get a new one
     unique_lock<mutex> lock(client->lock);
     io_mgr_->UnregisterContext(client->reader.get());
+    client->reader.reset();
     NewClient(client_id);
   }
 
@@ -170,11 +171,9 @@ void DiskIoMgrStress::ClientThread(int client_id) {
 // Cancel a random reader
 void DiskIoMgrStress::CancelRandomReader() {
   if (!includes_cancellation_) return;
-
-  int rand_client = rand() % num_clients_;
-
-  unique_lock<mutex> lock(clients_[rand_client].lock);
-  io_mgr_->CancelContext(clients_[rand_client].reader.get());
+  Client* rand_client = &clients_[rand() % num_clients_];
+  unique_lock<mutex> lock(rand_client->lock);
+  rand_client->reader->Cancel();
 }
 
 void DiskIoMgrStress::Run(int sec) {
@@ -199,10 +198,12 @@ void DiskIoMgrStress::Run(int sec) {
 
   for (int i = 0; i < num_clients_; ++i) {
     unique_lock<mutex> lock(clients_[i].lock);
-    if (clients_[i].reader != NULL) io_mgr_->CancelContext(clients_[i].reader.get());
+    if (clients_[i].reader != NULL) clients_[i].reader->Cancel();
   }
-
   readers_.join_all();
+
+  for (unique_ptr<MemTracker>& mem_tracker : client_mem_trackers_) mem_tracker->Close();
+  mem_tracker_.Close();
 }
 
 // Initialize a client to read one of the files at random.  The scan ranges are
@@ -240,6 +241,7 @@ void DiskIoMgrStress::NewClient(int i) {
     assigned_len += range_len;
   }
 
+  if (client_mem_trackers_[i] != nullptr) client_mem_trackers_[i]->Close();
   client_mem_trackers_[i].reset(new MemTracker(-1, "", &mem_tracker_));
   client.reader = io_mgr_->RegisterContext(client_mem_trackers_[i].get());
   Status status = io_mgr_->AddScanRanges(client.reader.get(), client.scan_ranges);

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/io/disk-io-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-test.cc b/be/src/runtime/io/disk-io-mgr-test.cc
index b03ec31..e099285 100644
--- a/be/src/runtime/io/disk-io-mgr-test.cc
+++ b/be/src/runtime/io/disk-io-mgr-test.cc
@@ -22,11 +22,13 @@
 
 #include "codegen/llvm-codegen.h"
 #include "common/init.h"
-#include "runtime/io/request-context.h"
 #include "runtime/io/disk-io-mgr-stress.h"
 #include "runtime/io/disk-io-mgr.h"
+#include "runtime/io/request-context.h"
 #include "runtime/mem-tracker.h"
+#include "runtime/test-env.h"
 #include "runtime/thread-resource-mgr.h"
+#include "service/fe-support.h"
 #include "testutil/gtest-util.h"
 #include "util/condition-variable.h"
 #include "util/cpu-info.h"
@@ -49,7 +51,10 @@ namespace io {
 class DiskIoMgrTest : public testing::Test {
  public:
 
-  virtual void SetUp() {}
+  virtual void SetUp() {
+    test_env_.reset(new TestEnv);
+    ASSERT_OK(test_env_->Init());
+  }
 
   virtual void TearDown() {
     pool_.Clear();
@@ -117,13 +122,14 @@ class DiskIoMgrTest : public testing::Test {
   static void ValidateSyncRead(DiskIoMgr* io_mgr, RequestContext* reader,
       ScanRange* range, const char* expected, int expected_len = -1) {
     unique_ptr<BufferDescriptor> buffer;
-    ASSERT_OK(io_mgr->Read(reader, range, &buffer));
+    ASSERT_OK(io_mgr->AddScanRange(reader, range, true));
+    ASSERT_OK(range->GetNext(&buffer));
     ASSERT_TRUE(buffer != nullptr);
     EXPECT_EQ(buffer->len(), range->len());
     if (expected_len < 0) expected_len = strlen(expected);
     int cmp = memcmp(buffer->buffer(), expected, expected_len);
     EXPECT_TRUE(cmp == 0);
-    io_mgr->ReturnBuffer(move(buffer));
+    range->ReturnBuffer(move(buffer));
   }
 
   static void ValidateScanRange(DiskIoMgr* io_mgr, ScanRange* range,
@@ -136,13 +142,13 @@ class DiskIoMgrTest : public testing::Test {
       Status status = range->GetNext(&buffer);
       ASSERT_TRUE(status.ok() || status.code() == expected_status.code());
       if (buffer == nullptr || !status.ok()) {
-        if (buffer != nullptr) io_mgr->ReturnBuffer(move(buffer));
+        if (buffer != nullptr) range->ReturnBuffer(move(buffer));
         break;
       }
       ASSERT_LE(buffer->len(), expected_len);
       memcpy(result + range->offset() + buffer->scan_range_offset(),
           buffer->buffer(), buffer->len());
-      io_mgr->ReturnBuffer(move(buffer));
+      range->ReturnBuffer(move(buffer));
     }
     ValidateEmptyOrCorrect(expected, result, expected_len);
   }
@@ -177,6 +183,8 @@ class DiskIoMgrTest : public testing::Test {
     return range;
   }
 
+  boost::scoped_ptr<TestEnv> test_env_;
+
   ObjectPool pool_;
 
   mutex written_mutex_;
@@ -204,14 +212,13 @@ TEST_F(DiskIoMgrTest, SingleWriter) {
 
   scoped_ptr<DiskIoMgr> read_io_mgr(new DiskIoMgr(1, 1, 1, 1, 10));
   MemTracker reader_mem_tracker(LARGE_MEM_LIMIT);
-  ASSERT_OK(read_io_mgr->Init(&reader_mem_tracker));
-  unique_ptr<RequestContext> reader =
-      read_io_mgr->RegisterContext(&reader_mem_tracker);
+  ASSERT_OK(read_io_mgr->Init());
+  unique_ptr<RequestContext> reader = read_io_mgr->RegisterContext(&reader_mem_tracker);
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
       pool_.Clear(); // Destroy scan ranges from previous iterations.
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 10);
-      ASSERT_OK(io_mgr.Init(&mem_tracker));
+      ASSERT_OK(io_mgr.Init());
       unique_ptr<RequestContext> writer = io_mgr.RegisterContext(&mem_tracker);
       for (int i = 0; i < num_ranges; ++i) {
         int32_t* data = pool_.Add(new int32_t);
@@ -247,7 +254,7 @@ TEST_F(DiskIoMgrTest, InvalidWrite) {
   num_ranges_written_ = 0;
   string tmp_file = "/non-existent/file.txt";
   DiskIoMgr io_mgr(1, 1, 1, 1, 10);
-  ASSERT_OK(io_mgr.Init(&mem_tracker));
+  ASSERT_OK(io_mgr.Init());
   unique_ptr<RequestContext> writer = io_mgr.RegisterContext(nullptr);
   int32_t* data = pool_.Add(new int32_t);
   *data = rand();
@@ -308,19 +315,18 @@ TEST_F(DiskIoMgrTest, SingleWriterCancel) {
 
   scoped_ptr<DiskIoMgr> read_io_mgr(new DiskIoMgr(1, 1, 1, 1, 10));
   MemTracker reader_mem_tracker(LARGE_MEM_LIMIT);
-  ASSERT_OK(read_io_mgr->Init(&reader_mem_tracker));
-  unique_ptr<RequestContext> reader =
-      read_io_mgr->RegisterContext(&reader_mem_tracker);
+  ASSERT_OK(read_io_mgr->Init());
+  unique_ptr<RequestContext> reader = read_io_mgr->RegisterContext(&reader_mem_tracker);
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
       pool_.Clear(); // Destroy scan ranges from previous iterations.
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 10);
-      ASSERT_OK(io_mgr.Init(&mem_tracker));
+      ASSERT_OK(io_mgr.Init());
       unique_ptr<RequestContext> writer = io_mgr.RegisterContext(&mem_tracker);
       Status validate_status = Status::OK();
       for (int i = 0; i < num_ranges; ++i) {
         if (i == num_ranges_before_cancel) {
-          io_mgr.CancelContext(writer.get());
+          writer->Cancel();
           validate_status = Status::CANCELLED;
         }
         int32_t* data = pool_.Add(new int32_t);
@@ -375,10 +381,9 @@ TEST_F(DiskIoMgrTest, SingleReader) {
         if (++iters % 5000 == 0) LOG(ERROR) << "Starting iteration " << iters;
         DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 1);
 
-        ASSERT_OK(io_mgr.Init(&mem_tracker));
+        ASSERT_OK(io_mgr.Init());
         MemTracker reader_mem_tracker;
-        unique_ptr<RequestContext> reader =
-            io_mgr.RegisterContext(&reader_mem_tracker);
+        unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
 
         vector<ScanRange*> ranges;
         for (int i = 0; i < len; ++i) {
@@ -426,10 +431,9 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) {
       if (++iters % 5000 == 0) LOG(ERROR) << "Starting iteration " << iters;
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 1);
 
-      ASSERT_OK(io_mgr.Init(&mem_tracker));
+      ASSERT_OK(io_mgr.Init());
       MemTracker reader_mem_tracker;
-      unique_ptr<RequestContext> reader =
-          io_mgr.RegisterContext(&reader_mem_tracker);
+      unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
 
       vector<ScanRange*> ranges_first_half;
       vector<ScanRange*> ranges_second_half;
@@ -496,11 +500,9 @@ TEST_F(DiskIoMgrTest, SyncReadTest) {
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk,
           MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
 
-      ASSERT_OK(io_mgr.Init(&mem_tracker));
+      ASSERT_OK(io_mgr.Init());
       MemTracker reader_mem_tracker;
-      unique_ptr<RequestContext> reader =
-          io_mgr.RegisterContext(&reader_mem_tracker);
-
+      unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
       ScanRange* complete_range =
           InitRange(tmp_file, 0, strlen(data), 0, stat_val.st_mtime);
 
@@ -563,10 +565,9 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) {
       if (++iters % 5000 == 0) LOG(ERROR) << "Starting iteration " << iters;
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 1);
 
-      ASSERT_OK(io_mgr.Init(&mem_tracker));
+      ASSERT_OK(io_mgr.Init());
       MemTracker reader_mem_tracker;
-      unique_ptr<RequestContext> reader =
-          io_mgr.RegisterContext(&reader_mem_tracker);
+      unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
 
       vector<ScanRange*> ranges;
       for (int i = 0; i < len; ++i) {
@@ -591,11 +592,11 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) {
             strlen(data), Status::CANCELLED, 0, &num_ranges_processed));
       }
 
-      io_mgr.CancelContext(reader.get());
+      reader->Cancel();
       sched_yield();
 
       threads.join_all();
-      EXPECT_TRUE(io_mgr.context_status(reader.get()).IsCancelled());
+      EXPECT_TRUE(reader->IsCancelled());
       io_mgr.UnregisterContext(reader.get());
       EXPECT_EQ(reader_mem_tracker.consumption(), 0);
     }
@@ -621,7 +622,7 @@ TEST_F(DiskIoMgrTest, MemLimits) {
     MemTracker root_mem_tracker(mem_limit_num_buffers * MAX_BUFFER_SIZE);
     DiskIoMgr io_mgr(1, 1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
 
-    ASSERT_OK(io_mgr.Init(&root_mem_tracker));
+    ASSERT_OK(io_mgr.Init());
     MemTracker reader_mem_tracker(-1, "Reader", &root_mem_tracker);
     unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
 
@@ -632,12 +633,13 @@ TEST_F(DiskIoMgrTest, MemLimits) {
     ASSERT_OK(io_mgr.AddScanRanges(reader.get(), ranges));
 
     // Don't return buffers to force memory pressure
-    vector<unique_ptr<BufferDescriptor>> buffers;
+    vector<pair<ScanRange*, unique_ptr<BufferDescriptor>>> buffers;
 
     AtomicInt32 num_ranges_processed;
     ScanRangeThread(&io_mgr, reader.get(), data, strlen(data), Status::MemLimitExceeded(),
         1, &num_ranges_processed);
 
+    bool hit_mem_limit_exceeded = false;
     char result[strlen(data) + 1];
     // Keep reading new ranges without returning buffers. This forces us
     // to go over the limit eventually.
@@ -646,25 +648,27 @@ TEST_F(DiskIoMgrTest, MemLimits) {
       ScanRange* range = nullptr;
       Status status = io_mgr.GetNextRange(reader.get(), &range);
       ASSERT_TRUE(status.ok() || status.IsMemLimitExceeded());
+      hit_mem_limit_exceeded |= status.IsMemLimitExceeded();
       if (range == nullptr) break;
 
       while (true) {
         unique_ptr<BufferDescriptor> buffer;
         Status status = range->GetNext(&buffer);
         ASSERT_TRUE(status.ok() || status.IsMemLimitExceeded());
+        hit_mem_limit_exceeded |= status.IsMemLimitExceeded();
         if (buffer == nullptr) break;
         memcpy(result + range->offset() + buffer->scan_range_offset(),
             buffer->buffer(), buffer->len());
-        buffers.push_back(move(buffer));
+        buffers.emplace_back(range, move(buffer));
       }
       ValidateEmptyOrCorrect(data, result, strlen(data));
     }
 
     for (int i = 0; i < buffers.size(); ++i) {
-      io_mgr.ReturnBuffer(move(buffers[i]));
+      buffers[i].first->ReturnBuffer(move(buffers[i].second));
     }
 
-    EXPECT_TRUE(io_mgr.context_status(reader.get()).IsMemLimitExceeded());
+    EXPECT_TRUE(hit_mem_limit_exceeded) << "Should have run out of memory";
     io_mgr.UnregisterContext(reader.get());
     EXPECT_EQ(reader_mem_tracker.consumption(), 0);
   }
@@ -689,7 +693,7 @@ TEST_F(DiskIoMgrTest, CachedReads) {
   {
     DiskIoMgr io_mgr(num_disks, 1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
 
-    ASSERT_OK(io_mgr.Init(&mem_tracker));
+    ASSERT_OK(io_mgr.Init());
     MemTracker reader_mem_tracker;
     unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
 
@@ -762,7 +766,7 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
       for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
         DiskIoMgr io_mgr(num_disks, threads_per_disk, threads_per_disk, MIN_BUFFER_SIZE,
             MAX_BUFFER_SIZE);
-        ASSERT_OK(io_mgr.Init(&mem_tracker));
+        ASSERT_OK(io_mgr.Init());
         for (int file_index = 0; file_index < num_contexts; ++file_index) {
           contexts[file_index] = io_mgr.RegisterContext(&mem_tracker);
         }
@@ -809,7 +813,6 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
             threads.join_all();
           } // for (int context_index
         } // while (read_offset < file_size)
-
         for (int file_index = 0; file_index < num_contexts; ++file_index) {
           io_mgr.UnregisterContext(contexts[file_index].get());
         }
@@ -874,7 +877,7 @@ TEST_F(DiskIoMgrTest, MultipleReader) {
 
         DiskIoMgr io_mgr(num_disks, threads_per_disk, threads_per_disk, MIN_BUFFER_SIZE,
             MAX_BUFFER_SIZE);
-        EXPECT_OK(io_mgr.Init(&mem_tracker));
+        ASSERT_OK(io_mgr.Init());
 
         for (int i = 0; i < NUM_READERS; ++i) {
           readers[i] = io_mgr.RegisterContext(&mem_tracker);
@@ -915,76 +918,6 @@ TEST_F(DiskIoMgrTest, StressTest) {
   test.Run(2); // In seconds
 }
 
-TEST_F(DiskIoMgrTest, Buffers) {
-  // Test default min/max buffer size
-  int min_buffer_size = 1024;
-  int max_buffer_size = 8 * 1024 * 1024; // 8 MB
-  MemTracker root_mem_tracker(max_buffer_size * 2);
-
-  DiskIoMgr io_mgr(1, 1, 1, min_buffer_size, max_buffer_size);
-  ASSERT_OK(io_mgr.Init(&root_mem_tracker));
-  ASSERT_EQ(root_mem_tracker.consumption(), 0);
-
-  MemTracker reader_mem_tracker(-1, "Reader", &root_mem_tracker);
-  unique_ptr<RequestContext> reader;
-  reader = io_mgr.RegisterContext(&reader_mem_tracker);
-
-  ScanRange* dummy_range = InitRange("dummy", 0, 0, 0, 0);
-
-  // buffer length should be rounded up to min buffer size
-  int64_t buffer_len = 1;
-  unique_ptr<BufferDescriptor> buffer_desc;
-  buffer_desc = io_mgr.GetFreeBuffer(reader.get(), dummy_range, buffer_len);
-  EXPECT_TRUE(buffer_desc->buffer() != nullptr);
-  EXPECT_EQ(min_buffer_size, buffer_desc->buffer_len());
-  EXPECT_EQ(1, io_mgr.num_allocated_buffers_.Load());
-  io_mgr.FreeBufferMemory(buffer_desc.get());
-  io_mgr.ReturnBuffer(move(buffer_desc));
-  EXPECT_EQ(min_buffer_size, root_mem_tracker.consumption());
-
-  // reuse buffer
-  buffer_len = min_buffer_size;
-  buffer_desc = io_mgr.GetFreeBuffer(reader.get(), dummy_range, buffer_len);
-  EXPECT_TRUE(buffer_desc->buffer() != nullptr);
-  EXPECT_EQ(min_buffer_size, buffer_desc->buffer_len());
-  EXPECT_EQ(1, io_mgr.num_allocated_buffers_.Load());
-  io_mgr.FreeBufferMemory(buffer_desc.get());
-  io_mgr.ReturnBuffer(move(buffer_desc));
-  EXPECT_EQ(min_buffer_size, root_mem_tracker.consumption());
-
-  // bump up to next buffer size
-  buffer_len = min_buffer_size + 1;
-  buffer_desc = io_mgr.GetFreeBuffer(reader.get(), dummy_range, buffer_len);
-  EXPECT_TRUE(buffer_desc->buffer() != nullptr);
-  EXPECT_EQ(min_buffer_size * 2, buffer_desc->buffer_len());
-  EXPECT_EQ(2, io_mgr.num_allocated_buffers_.Load());
-  EXPECT_EQ(min_buffer_size * 3, root_mem_tracker.consumption());
-
-  // gc unused buffer
-  io_mgr.GcIoBuffers();
-  EXPECT_EQ(1, io_mgr.num_allocated_buffers_.Load());
-  EXPECT_EQ(min_buffer_size * 2, root_mem_tracker.consumption());
-
-  io_mgr.FreeBufferMemory(buffer_desc.get());
-  io_mgr.ReturnBuffer(move(buffer_desc));
-
-  // max buffer size
-  buffer_len = max_buffer_size;
-  buffer_desc = io_mgr.GetFreeBuffer(reader.get(), dummy_range, buffer_len);
-  EXPECT_TRUE(buffer_desc->buffer() != nullptr);
-  EXPECT_EQ(max_buffer_size, buffer_desc->buffer_len());
-  EXPECT_EQ(2, io_mgr.num_allocated_buffers_.Load());
-  io_mgr.FreeBufferMemory(buffer_desc.get());
-  io_mgr.ReturnBuffer(move(buffer_desc));
-  EXPECT_EQ(min_buffer_size * 2 + max_buffer_size, root_mem_tracker.consumption());
-
-  // gc buffers
-  io_mgr.GcIoBuffers();
-  EXPECT_EQ(io_mgr.num_allocated_buffers_.Load(), 0);
-  EXPECT_EQ(root_mem_tracker.consumption(), 0);
-  io_mgr.UnregisterContext(reader.get());
-}
-
 // IMPALA-2366: handle partial read where range goes past end of file.
 TEST_F(DiskIoMgrTest, PartialRead) {
   MemTracker mem_tracker(LARGE_MEM_LIMIT);
@@ -1000,7 +933,7 @@ TEST_F(DiskIoMgrTest, PartialRead) {
 
   scoped_ptr<DiskIoMgr> io_mgr(new DiskIoMgr(1, 1, 1, read_len, read_len));
 
-  ASSERT_OK(io_mgr->Init(&mem_tracker));
+  ASSERT_OK(io_mgr->Init());
   MemTracker reader_mem_tracker;
   unique_ptr<RequestContext> reader;
   reader = io_mgr->RegisterContext(&reader_mem_tracker);
@@ -1008,11 +941,12 @@ TEST_F(DiskIoMgrTest, PartialRead) {
   // We should not read past the end of file.
   ScanRange* range = InitRange(tmp_file, 0, read_len, 0, stat_val.st_mtime);
   unique_ptr<BufferDescriptor> buffer;
-  ASSERT_OK(io_mgr->Read(reader.get(), range, &buffer));
+  ASSERT_OK(io_mgr->AddScanRange(reader.get(), range, true));
+  ASSERT_OK(range->GetNext(&buffer));
   ASSERT_TRUE(buffer->eosr());
   ASSERT_EQ(len, buffer->len());
   ASSERT_TRUE(memcmp(buffer->buffer(), data, len) == 0);
-  io_mgr->ReturnBuffer(move(buffer));
+  range->ReturnBuffer(move(buffer));
 
   io_mgr->UnregisterContext(reader.get());
   pool_.Clear();
@@ -1032,7 +966,7 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBuffer) {
 
   scoped_ptr<DiskIoMgr> io_mgr(new DiskIoMgr(1, 1, 1, read_len, read_len));
 
-  ASSERT_OK(io_mgr->Init(&mem_tracker));
+  ASSERT_OK(io_mgr->Init());
   // Reader doesn't need to provide mem tracker if it's providing buffers.
   MemTracker* reader_mem_tracker = nullptr;
   unique_ptr<RequestContext> reader;
@@ -1055,7 +989,7 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBuffer) {
 
     // DiskIoMgr should not have allocated memory.
     EXPECT_EQ(mem_tracker.consumption(), 0);
-    io_mgr->ReturnBuffer(move(io_buffer));
+    range->ReturnBuffer(move(io_buffer));
   }
 
   io_mgr->UnregisterContext(reader.get());
@@ -1072,7 +1006,7 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBufferError) {
 
   scoped_ptr<DiskIoMgr> io_mgr(new DiskIoMgr(1, 1, 1, SCAN_LEN, SCAN_LEN));
 
-  ASSERT_OK(io_mgr->Init(&mem_tracker));
+  ASSERT_OK(io_mgr->Init());
   // Reader doesn't need to provide mem tracker if it's providing buffers.
   MemTracker* reader_mem_tracker = nullptr;
   unique_ptr<RequestContext> reader;
@@ -1086,7 +1020,7 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBufferError) {
 
     /// Also test the cancellation path. Run multiple iterations since it is racy whether
     /// the read fails before the cancellation.
-    if (i >= 1) io_mgr->CancelContext(reader.get());
+    if (i >= 1) reader->Cancel();
 
     unique_ptr<BufferDescriptor> io_buffer;
     ASSERT_FALSE(range->GetNext(&io_buffer).ok());
@@ -1114,7 +1048,7 @@ TEST_F(DiskIoMgrTest, VerifyNumThreadsParameter) {
   const int num_io_threads_per_rotational_or_ssd = 2;
   DiskIoMgr io_mgr(1, num_io_threads_per_rotational_or_ssd,
       num_io_threads_per_rotational_or_ssd, 1, 10);
-  ASSERT_OK(io_mgr.Init(&mem_tracker));
+  ASSERT_OK(io_mgr.Init());
   const int num_io_threads = io_mgr.disk_thread_group_.Size();
   ASSERT_TRUE(num_io_threads ==
       num_io_threads_per_rotational_or_ssd + num_io_threads_for_remote_disks);
@@ -1125,5 +1059,6 @@ TEST_F(DiskIoMgrTest, VerifyNumThreadsParameter) {
 int main(int argc, char** argv) {
   ::testing::InitGoogleTest(&argc, argv);
   impala::InitCommonRuntime(argc, argv, true, impala::TestInfo::BE_TEST);
+  impala::InitFeSupport();
   return RUN_ALL_TESTS();
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/io/disk-io-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr.cc b/be/src/runtime/io/disk-io-mgr.cc
index f3d69f1..8c00ef8 100644
--- a/be/src/runtime/io/disk-io-mgr.cc
+++ b/be/src/runtime/io/disk-io-mgr.cc
@@ -157,34 +157,6 @@ string DiskIoMgr::DebugString() {
   return ss.str();
 }
 
-BufferDescriptor::BufferDescriptor(DiskIoMgr* io_mgr,
-    RequestContext* reader, ScanRange* scan_range, uint8_t* buffer,
-    int64_t buffer_len, MemTracker* mem_tracker)
-  : io_mgr_(io_mgr),
-    reader_(reader),
-    mem_tracker_(mem_tracker),
-    scan_range_(scan_range),
-    buffer_(buffer),
-    buffer_len_(buffer_len) {
-  DCHECK(io_mgr != nullptr);
-  DCHECK(scan_range != nullptr);
-  DCHECK(buffer != nullptr);
-  DCHECK_GE(buffer_len, 0);
-  DCHECK_NE(scan_range->external_buffer_tag_ == ScanRange::ExternalBufferTag::NO_BUFFER,
-      mem_tracker == nullptr);
-}
-
-void BufferDescriptor::TransferOwnership(MemTracker* dst) {
-  DCHECK(dst != nullptr);
-  DCHECK(!is_client_buffer());
-  // Memory of cached buffers is not tracked against a tracker.
-  if (is_cached()) return;
-  DCHECK(mem_tracker_ != nullptr);
-  dst->Consume(buffer_len_);
-  mem_tracker_->Release(buffer_len_);
-  mem_tracker_ = dst;
-}
-
 WriteRange::WriteRange(
     const string& file, int64_t file_offset, int disk_id, WriteDoneCallback callback)
   : RequestRange(RequestType::WRITE), callback_(callback) {
@@ -235,8 +207,6 @@ DiskIoMgr::DiskIoMgr() :
         FLAGS_num_file_handle_cache_partitions,
         FLAGS_unused_file_handle_timeout_sec) {
   DCHECK_LE(READ_SIZE_MIN_VALUE, FLAGS_read_size);
-  int64_t max_buffer_size_scaled = BitUtil::Ceil(max_buffer_size_, min_buffer_size_);
-  free_buffers_.resize(BitUtil::Log2Ceiling64(max_buffer_size_scaled) + 1);
   int num_local_disks = DiskInfo::num_disks();
   if (FLAGS_num_disks < 0 || FLAGS_num_disks > DiskInfo::num_disks()) {
     LOG(WARNING) << "Number of disks specified should be between 0 and the number of "
@@ -250,7 +220,7 @@ DiskIoMgr::DiskIoMgr() :
 }
 
 DiskIoMgr::DiskIoMgr(int num_local_disks, int threads_per_rotational_disk,
-    int threads_per_solid_state_disk, int min_buffer_size, int max_buffer_size) :
+    int threads_per_solid_state_disk, int64_t min_buffer_size, int64_t max_buffer_size) :
     num_io_threads_per_rotational_disk_(threads_per_rotational_disk),
     num_io_threads_per_solid_state_disk_(threads_per_solid_state_disk),
     max_buffer_size_(max_buffer_size),
@@ -262,8 +232,6 @@ DiskIoMgr::DiskIoMgr(int num_local_disks, int threads_per_rotational_disk,
         FileSystemUtil::MaxNumFileHandles()),
         FLAGS_num_file_handle_cache_partitions,
         FLAGS_unused_file_handle_timeout_sec) {
-  int64_t max_buffer_size_scaled = BitUtil::Ceil(max_buffer_size_, min_buffer_size_);
-  free_buffers_.resize(BitUtil::Log2Ceiling64(max_buffer_size_scaled) + 1);
   if (num_local_disks == 0) num_local_disks = DiskInfo::num_disks();
   disk_queues_.resize(num_local_disks + REMOTE_NUM_DISKS);
   CheckSseSupport();
@@ -287,37 +255,22 @@ DiskIoMgr::~DiskIoMgr() {
   for (int i = 0; i < disk_queues_.size(); ++i) {
     if (disk_queues_[i] == nullptr) continue;
     int disk_id = disk_queues_[i]->disk_id;
-    for (list<RequestContext*>::iterator it = disk_queues_[i]->request_contexts.begin();
-        it != disk_queues_[i]->request_contexts.end(); ++it) {
-      DCHECK_EQ((*it)->disk_states_[disk_id].num_threads_in_op(), 0);
-      DCHECK((*it)->disk_states_[disk_id].done());
-      (*it)->DecrementDiskRefCount();
+    for (RequestContext* context : disk_queues_[i]->request_contexts) {
+      unique_lock<mutex> context_lock(context->lock_);
+      DCHECK_EQ(context->disk_states_[disk_id].num_threads_in_op(), 0);
+      DCHECK(context->disk_states_[disk_id].done());
+      context->DecrementDiskRefCount(context_lock);
     }
   }
 
-  DCHECK_EQ(num_buffers_in_readers_.Load(), 0);
-
-  // Delete all allocated buffers
-  int num_free_buffers = 0;
-  for (int idx = 0; idx < free_buffers_.size(); ++idx) {
-    num_free_buffers += free_buffers_[idx].size();
-  }
-  DCHECK_EQ(num_allocated_buffers_.Load(), num_free_buffers);
-  GcIoBuffers();
-
   for (int i = 0; i < disk_queues_.size(); ++i) {
     delete disk_queues_[i];
   }
 
-  if (free_buffer_mem_tracker_ != nullptr) free_buffer_mem_tracker_->Close();
   if (cached_read_options_ != nullptr) hadoopRzOptionsFree(cached_read_options_);
 }
 
-Status DiskIoMgr::Init(MemTracker* process_mem_tracker) {
-  DCHECK(process_mem_tracker != nullptr);
-  free_buffer_mem_tracker_.reset(
-      new MemTracker(-1, "Free Disk IO Buffers", process_mem_tracker, false));
-
+Status DiskIoMgr::Init() {
   for (int i = 0; i < disk_queues_.size(); ++i) {
     disk_queues_[i] = new DiskQueue(i);
     int num_threads_per_disk;
@@ -372,88 +325,6 @@ void DiskIoMgr::UnregisterContext(RequestContext* reader) {
   reader->CancelAndMarkInactive();
 }
 
-// Cancellation requires coordination from multiple threads.  Each thread that currently
-// has a reference to the request context must notice the cancel and remove it from its
-// tracking structures.  The last thread to touch the context should deallocate (aka
-// recycle) the request context object.  Potential threads are:
-//  1. Disk threads that are currently reading for this reader.
-//  2. Caller threads that are waiting in GetNext.
-//
-// The steps are:
-// 1. Cancel will immediately set the context in the Cancelled state.  This prevents any
-// other thread from adding more ready buffers to the context (they all take a lock and
-// check the state before doing so), or any write ranges to the context.
-// 2. Cancel will call cancel on each ScanRange that is not yet complete, unblocking
-// any threads in GetNext(). The reader will see the cancelled Status returned. Cancel
-// also invokes the callback for the WriteRanges with the cancelled state.
-// 3. Disk threads notice the context is cancelled either when picking the next context
-// to process or when they try to enqueue a ready buffer.  Upon noticing the cancelled
-// state, removes the context from the disk queue.  The last thread per disk with an
-// outstanding reference to the context decrements the number of disk queues the context
-// is on.
-void DiskIoMgr::CancelContext(RequestContext* context) {
-  context->Cancel(Status::CANCELLED);
-}
-
-void DiskIoMgr::set_read_timer(RequestContext* r, RuntimeProfile::Counter* c) {
-  r->read_timer_ = c;
-}
-
-void DiskIoMgr::set_bytes_read_counter(RequestContext* r, RuntimeProfile::Counter* c) {
-  r->bytes_read_counter_ = c;
-}
-
-void DiskIoMgr::set_active_read_thread_counter(RequestContext* r,
-    RuntimeProfile::Counter* c) {
-  r->active_read_thread_counter_ = c;
-}
-
-void DiskIoMgr::set_disks_access_bitmap(RequestContext* r,
-    RuntimeProfile::Counter* c) {
-  r->disks_accessed_bitmap_ = c;
-}
-
-int64_t DiskIoMgr::queue_size(RequestContext* reader) const {
-  return reader->num_ready_buffers_.Load();
-}
-
-Status DiskIoMgr::context_status(RequestContext* context) const {
-  unique_lock<mutex> lock(context->lock_);
-  return context->status_;
-}
-
-int64_t DiskIoMgr::bytes_read_local(RequestContext* reader) const {
-  return reader->bytes_read_local_.Load();
-}
-
-int64_t DiskIoMgr::bytes_read_short_circuit(RequestContext* reader) const {
-  return reader->bytes_read_short_circuit_.Load();
-}
-
-int64_t DiskIoMgr::bytes_read_dn_cache(RequestContext* reader) const {
-  return reader->bytes_read_dn_cache_.Load();
-}
-
-int DiskIoMgr::num_remote_ranges(RequestContext* reader) const {
-  return reader->num_remote_ranges_.Load();
-}
-
-int64_t DiskIoMgr::unexpected_remote_bytes(RequestContext* reader) const {
-  return reader->unexpected_remote_bytes_.Load();
-}
-
-int DiskIoMgr::cached_file_handles_hit_count(RequestContext* reader) const {
-  return reader->cached_file_handles_hit_count_.Load();
-}
-
-int DiskIoMgr::cached_file_handles_miss_count(RequestContext* reader) const {
-  return reader->cached_file_handles_miss_count_.Load();
-}
-
-int64_t DiskIoMgr::GetReadThroughput() {
-  return RuntimeProfile::UnitsPerSecond(&total_bytes_read_counter_, &read_timer_);
-}
-
 Status DiskIoMgr::ValidateScanRange(ScanRange* range) {
   int disk_id = range->disk_id_;
   if (disk_id < 0 || disk_id >= disk_queues_.size()) {
@@ -485,10 +356,7 @@ Status DiskIoMgr::AddScanRanges(RequestContext* reader,
   unique_lock<mutex> reader_lock(reader->lock_);
   DCHECK(reader->Validate()) << endl << reader->DebugString();
 
-  if (reader->state_ == RequestContext::Cancelled) {
-    DCHECK(!reader->status_.ok());
-    return reader->status_;
-  }
+  if (reader->state_ == RequestContext::Cancelled) return Status::CANCELLED;
 
   // Add each range to the queue of the disk the range is on
   for (int i = 0; i < ranges.size(); ++i) {
@@ -507,7 +375,7 @@ Status DiskIoMgr::AddScanRanges(RequestContext* reader,
         continue;
       }
     }
-    reader->AddRequestRange(range, schedule_immediately);
+    reader->AddRequestRange(reader_lock, range, schedule_immediately);
   }
   DCHECK(reader->Validate()) << endl << reader->DebugString();
 
@@ -533,8 +401,7 @@ Status DiskIoMgr::GetNextRange(RequestContext* reader, ScanRange** range) {
 
   while (true) {
     if (reader->state_ == RequestContext::Cancelled) {
-      DCHECK(!reader->status_.ok());
-      status = reader->status_;
+      status = Status::CANCELLED;
       break;
     }
 
@@ -553,7 +420,7 @@ Status DiskIoMgr::GetNextRange(RequestContext* reader, ScanRange** range) {
       if (cached_read_succeeded) return Status::OK();
 
       // This range ended up not being cached. Loop again and pick up a new range.
-      reader->AddRequestRange(*range, false);
+      reader->AddRequestRange(reader_lock, *range, false);
       DCHECK(reader->Validate()) << endl << reader->DebugString();
       *range = nullptr;
       continue;
@@ -569,185 +436,13 @@ Status DiskIoMgr::GetNextRange(RequestContext* reader, ScanRange** range) {
       // Set this to nullptr, the next time this disk runs for this reader, it will
       // get another range ready.
       reader->disk_states_[disk_id].set_next_scan_range_to_start(nullptr);
-      reader->ScheduleScanRange(*range);
+      reader->ScheduleScanRange(reader_lock, *range);
       break;
     }
   }
   return status;
 }
 
-Status DiskIoMgr::Read(RequestContext* reader,
-    ScanRange* range, std::unique_ptr<BufferDescriptor>* buffer) {
-  DCHECK(range != nullptr);
-  DCHECK(buffer != nullptr);
-  *buffer = nullptr;
-
-  if (range->len() > max_buffer_size_
-      && range->external_buffer_tag_ != ScanRange::ExternalBufferTag::CLIENT_BUFFER) {
-    return Status(TErrorCode::DISK_IO_ERROR, Substitute("Internal error: cannot "
-        "perform sync read of '$0' bytes that is larger than the max read buffer size "
-        "'$1'.", range->len(), max_buffer_size_));
-  }
-
-  vector<ScanRange*> ranges;
-  ranges.push_back(range);
-  RETURN_IF_ERROR(AddScanRanges(reader, ranges, true));
-  RETURN_IF_ERROR(range->GetNext(buffer));
-  DCHECK((*buffer) != nullptr);
-  DCHECK((*buffer)->eosr());
-  return Status::OK();
-}
-
-void DiskIoMgr::ReturnBuffer(unique_ptr<BufferDescriptor> buffer_desc) {
-  DCHECK(buffer_desc != nullptr);
-  if (!buffer_desc->status_.ok()) DCHECK(buffer_desc->buffer_ == nullptr);
-
-  RequestContext* reader = buffer_desc->reader_;
-  if (buffer_desc->buffer_ != nullptr) {
-    if (!buffer_desc->is_cached() && !buffer_desc->is_client_buffer()) {
-      // Buffers the were not allocated by DiskIoMgr don't need to be freed.
-      FreeBufferMemory(buffer_desc.get());
-    }
-    buffer_desc->buffer_ = nullptr;
-    num_buffers_in_readers_.Add(-1);
-    reader->num_buffers_in_reader_.Add(-1);
-  } else {
-    // A nullptr buffer means there was an error in which case there is no buffer
-    // to return.
-  }
-
-  if (buffer_desc->eosr_ || buffer_desc->scan_range_->is_cancelled_) {
-    // Need to close the scan range if returning the last buffer or the scan range
-    // has been cancelled (and the caller might never get the last buffer).
-    // Close() is idempotent so multiple cancelled buffers is okay.
-    buffer_desc->scan_range_->Close();
-  }
-}
-
-unique_ptr<BufferDescriptor> DiskIoMgr::GetFreeBuffer(
-    RequestContext* reader, ScanRange* range, int64_t buffer_size) {
-  DCHECK_LE(buffer_size, max_buffer_size_);
-  DCHECK_GT(buffer_size, 0);
-  buffer_size = min(static_cast<int64_t>(max_buffer_size_), buffer_size);
-  int idx = free_buffers_idx(buffer_size);
-  // Quantize buffer size to nearest power of 2 greater than the specified buffer size and
-  // convert to bytes
-  buffer_size = (1LL << idx) * min_buffer_size_;
-
-  // Track memory against the reader. This is checked the next time we start
-  // a read for the next reader in DiskIoMgr::GetNextScanRange().
-  DCHECK(reader->mem_tracker_ != nullptr);
-  reader->mem_tracker_->Consume(buffer_size);
-
-  uint8_t* buffer = nullptr;
-  {
-    unique_lock<mutex> lock(free_buffers_lock_);
-    if (free_buffers_[idx].empty()) {
-      num_allocated_buffers_.Add(1);
-      if (ImpaladMetrics::IO_MGR_NUM_BUFFERS != nullptr) {
-        ImpaladMetrics::IO_MGR_NUM_BUFFERS->Increment(1L);
-      }
-      if (ImpaladMetrics::IO_MGR_TOTAL_BYTES != nullptr) {
-        ImpaladMetrics::IO_MGR_TOTAL_BYTES->Increment(buffer_size);
-      }
-      // We already tracked this memory against the reader's MemTracker.
-      buffer = new uint8_t[buffer_size];
-    } else {
-      if (ImpaladMetrics::IO_MGR_NUM_UNUSED_BUFFERS != nullptr) {
-        ImpaladMetrics::IO_MGR_NUM_UNUSED_BUFFERS->Increment(-1L);
-      }
-      buffer = free_buffers_[idx].front();
-      free_buffers_[idx].pop_front();
-      free_buffer_mem_tracker_->Release(buffer_size);
-      ASAN_UNPOISON_MEMORY_REGION(buffer, buffer_size);
-    }
-  }
-
-  // Validate more invariants.
-  DCHECK(range != nullptr);
-  DCHECK(reader != nullptr);
-  DCHECK(buffer != nullptr);
-  return unique_ptr<BufferDescriptor>(new BufferDescriptor(
-      this, reader, range, buffer, buffer_size, reader->mem_tracker_));
-}
-
-void DiskIoMgr::GcIoBuffers(int64_t bytes_to_free) {
-  unique_lock<mutex> lock(free_buffers_lock_);
-  int buffers_freed = 0;
-  int bytes_freed = 0;
-  // Free small-to-large to avoid retaining many small buffers and fragmenting memory.
-  for (int idx = 0; idx < free_buffers_.size(); ++idx) {
-    deque<uint8_t*>* free_buffers = &free_buffers_[idx];
-    while (
-        !free_buffers->empty() && (bytes_to_free == -1 || bytes_freed <= bytes_to_free)) {
-      uint8_t* buffer = free_buffers->front();
-      free_buffers->pop_front();
-      int64_t buffer_size = (1LL << idx) * min_buffer_size_;
-      ASAN_UNPOISON_MEMORY_REGION(buffer, buffer_size);
-      delete[] buffer;
-      free_buffer_mem_tracker_->Release(buffer_size);
-      num_allocated_buffers_.Add(-1);
-
-      ++buffers_freed;
-      bytes_freed += buffer_size;
-    }
-    if (bytes_to_free != -1 && bytes_freed >= bytes_to_free) break;
-  }
-
-  if (ImpaladMetrics::IO_MGR_NUM_BUFFERS != nullptr) {
-    ImpaladMetrics::IO_MGR_NUM_BUFFERS->Increment(-buffers_freed);
-  }
-  if (ImpaladMetrics::IO_MGR_TOTAL_BYTES != nullptr) {
-    ImpaladMetrics::IO_MGR_TOTAL_BYTES->Increment(-bytes_freed);
-  }
-  if (ImpaladMetrics::IO_MGR_NUM_UNUSED_BUFFERS != nullptr) {
-    ImpaladMetrics::IO_MGR_NUM_UNUSED_BUFFERS->Increment(-buffers_freed);
-  }
-}
-
-void DiskIoMgr::FreeBufferMemory(BufferDescriptor* desc) {
-  DCHECK(!desc->is_cached());
-  DCHECK(!desc->is_client_buffer());
-  uint8_t* buffer = desc->buffer_;
-  int64_t buffer_size = desc->buffer_len_;
-  int idx = free_buffers_idx(buffer_size);
-  DCHECK_EQ(BitUtil::Ceil(buffer_size, min_buffer_size_) & ~(1LL << idx), 0)
-      << "buffer_size_ / min_buffer_size_ should be power of 2, got buffer_size = "
-      << buffer_size << ", min_buffer_size_ = " << min_buffer_size_;
-
-  {
-    unique_lock<mutex> lock(free_buffers_lock_);
-    if (!FLAGS_disable_mem_pools &&
-        free_buffers_[idx].size() < FLAGS_max_free_io_buffers) {
-      // Poison buffers stored in cache.
-      ASAN_POISON_MEMORY_REGION(buffer, buffer_size);
-      free_buffers_[idx].push_back(buffer);
-      if (ImpaladMetrics::IO_MGR_NUM_UNUSED_BUFFERS != nullptr) {
-        ImpaladMetrics::IO_MGR_NUM_UNUSED_BUFFERS->Increment(1L);
-      }
-      // This consume call needs to be protected by 'free_buffers_lock_' to avoid a race
-      // with a Release() call for the same buffer that could make consumption negative.
-      // Note: we can't use TryConsume(), which can indirectly call GcIoBuffers().
-      // TODO: after IMPALA-3200 is completed, we should be able to leverage the buffer
-      // pool's free lists, and remove these free lists.
-      free_buffer_mem_tracker_->Consume(buffer_size);
-    } else {
-      num_allocated_buffers_.Add(-1);
-      delete[] buffer;
-      if (ImpaladMetrics::IO_MGR_NUM_BUFFERS != nullptr) {
-        ImpaladMetrics::IO_MGR_NUM_BUFFERS->Increment(-1L);
-      }
-      if (ImpaladMetrics::IO_MGR_TOTAL_BYTES != nullptr) {
-        ImpaladMetrics::IO_MGR_TOTAL_BYTES->Increment(-buffer_size);
-      }
-    }
-  }
-
-  // We transferred the buffer ownership from the BufferDescriptor to the DiskIoMgr.
-  desc->mem_tracker_->Release(buffer_size);
-  desc->buffer_ = nullptr;
-}
-
 // This function gets the next RequestRange to work on for this disk. It checks for
 // cancellation and
 // a) Updates ready_to_start_ranges if there are no scan ranges queued for this disk.
@@ -785,7 +480,7 @@ bool DiskIoMgr::GetNextRequestRange(DiskQueue* disk_queue, RequestRange** range,
       disk_queue->request_contexts.pop_front();
       DCHECK(*request_context != nullptr);
       request_disk_state = &((*request_context)->disk_states_[disk_id]);
-      request_disk_state->IncrementRequestThreadAndDequeue();
+      request_disk_state->IncrementDiskThreadAndDequeue();
     }
 
     // NOTE: no locks were taken in between.  We need to be careful about what state
@@ -793,27 +488,13 @@ bool DiskIoMgr::GetNextRequestRange(DiskQueue* disk_queue, RequestRange** range,
     // There are some invariants here.  Only one disk thread can have the
     // same reader here (the reader is removed from the queue).  There can be
     // other disk threads operating on this reader in other functions though.
-
-    // We just picked a reader. Before we may allocate a buffer on its behalf, check that
-    // it has not exceeded any memory limits (e.g. the query or process limit).
-    // TODO: once IMPALA-3200 is fixed, we should be able to remove the free lists and
-    // move these memory limit checks to GetFreeBuffer().
-    // Note that calling AnyLimitExceeded() can result in a call to GcIoBuffers().
-    // TODO: IMPALA-3209: we should not force a reader over its memory limit by
-    // pushing more buffers to it. Most readers can make progress and operate within
-    // a fixed memory limit.
-    if ((*request_context)->mem_tracker_ != nullptr
-        && (*request_context)->mem_tracker_->AnyLimitExceeded()) {
-      (*request_context)->Cancel(Status::MemLimitExceeded());
-    }
-
     unique_lock<mutex> request_lock((*request_context)->lock_);
     VLOG_FILE << "Disk (id=" << disk_id << ") reading for "
         << (*request_context)->DebugString();
 
     // Check if reader has been cancelled
     if ((*request_context)->state_ == RequestContext::Cancelled) {
-      request_disk_state->DecrementRequestThreadAndCheckDone(*request_context);
+      request_disk_state->DecrementDiskThread(request_lock, *request_context);
       continue;
     }
 
@@ -856,7 +537,7 @@ bool DiskIoMgr::GetNextRequestRange(DiskQueue* disk_queue, RequestRange** range,
 
     // There are no inflight ranges, nothing to do.
     if (request_disk_state->in_flight_ranges()->empty()) {
-      request_disk_state->DecrementRequestThread();
+      request_disk_state->DecrementDiskThread(request_lock, *request_context);
       continue;
     }
     DCHECK_GT(request_disk_state->num_remaining_ranges(), 0);
@@ -865,7 +546,7 @@ bool DiskIoMgr::GetNextRequestRange(DiskQueue* disk_queue, RequestRange** range,
 
     // Now that we've picked a request range, put the context back on the queue so
     // another thread can pick up another request range for this context.
-    request_disk_state->ScheduleContext(*request_context, disk_id);
+    request_disk_state->ScheduleContext(request_lock, *request_context, disk_id);
     DCHECK((*request_context)->Validate()) << endl << (*request_context)->DebugString();
     return true;
   }
@@ -879,81 +560,67 @@ void DiskIoMgr::HandleWriteFinished(
   // Copy disk_id before running callback: the callback may modify write_range.
   int disk_id = write_range->disk_id_;
 
-  // Execute the callback before decrementing the thread count. Otherwise CancelContext()
-  // that waits for the disk ref count to be 0 will return, creating a race, e.g. see
-  // IMPALA-1890.
+  // Execute the callback before decrementing the thread count. Otherwise
+  // RequestContext::Cancel() that waits for the disk ref count to be 0 will
+  // return, creating a race, e.g. see IMPALA-1890.
   // The status of the write does not affect the status of the writer context.
   write_range->callback_(write_status);
   {
     unique_lock<mutex> writer_lock(writer->lock_);
     DCHECK(writer->Validate()) << endl << writer->DebugString();
     RequestContext::PerDiskState& state = writer->disk_states_[disk_id];
-    if (writer->state_ == RequestContext::Cancelled) {
-      state.DecrementRequestThreadAndCheckDone(writer);
-    } else {
-      state.DecrementRequestThread();
-    }
+    state.DecrementDiskThread(writer_lock, writer);
     --state.num_remaining_ranges();
   }
 }
 
 void DiskIoMgr::HandleReadFinished(DiskQueue* disk_queue, RequestContext* reader,
-    unique_ptr<BufferDescriptor> buffer) {
+    Status read_status, unique_ptr<BufferDescriptor> buffer) {
   unique_lock<mutex> reader_lock(reader->lock_);
 
-  RequestContext::PerDiskState& state = reader->disk_states_[disk_queue->disk_id];
+  RequestContext::PerDiskState* disk_state = &reader->disk_states_[disk_queue->disk_id];
   DCHECK(reader->Validate()) << endl << reader->DebugString();
-  DCHECK_GT(state.num_threads_in_op(), 0);
-  DCHECK(buffer->buffer_ != nullptr);
-
-  if (reader->state_ == RequestContext::Cancelled) {
-    state.DecrementRequestThreadAndCheckDone(reader);
-    DCHECK(reader->Validate()) << endl << reader->DebugString();
-    if (!buffer->is_client_buffer()) FreeBufferMemory(buffer.get());
-    buffer->buffer_ = nullptr;
-    ScanRange* scan_range = buffer->scan_range_;
-    scan_range->Cancel(reader->status_);
-    // Enqueue the buffer to use the scan range's buffer cleanup path.
-    scan_range->EnqueueBuffer(reader_lock, move(buffer));
-    return;
-  }
-
-  DCHECK_EQ(reader->state_, RequestContext::Active);
+  DCHECK_GT(disk_state->num_threads_in_op(), 0);
   DCHECK(buffer->buffer_ != nullptr);
+  DCHECK(!buffer->is_cached()) << "HDFS cache reads don't go through this code path.";
 
-  // Update the reader's scan ranges.  There are a three cases here:
-  //  1. Read error
-  //  2. End of scan range
-  //  3. Middle of scan range
-  if (!buffer->status_.ok()) {
-    // Error case
-    if (!buffer->is_client_buffer()) FreeBufferMemory(buffer.get());
-    buffer->buffer_ = nullptr;
-    buffer->eosr_ = true;
-    --state.num_remaining_ranges();
-    buffer->scan_range_->Cancel(buffer->status_);
-  } else if (buffer->eosr_) {
-    --state.num_remaining_ranges();
-  }
-
-  // After calling EnqueueBuffer(), it is no longer valid to read from buffer.
+  // After calling EnqueueBuffer() below, it is no longer valid to read from buffer.
   // Store the state we need before calling EnqueueBuffer().
   bool eosr = buffer->eosr_;
+
+  // TODO: IMPALA-4249: it safe to touch 'scan_range' until DecrementDiskThread() is
+  // called because all clients of DiskIoMgr keep ScanRange objects alive until they
+  // unregister their RequestContext.
   ScanRange* scan_range = buffer->scan_range_;
-  bool is_cached = buffer->is_cached();
-  bool queue_full = scan_range->EnqueueBuffer(reader_lock, move(buffer));
-  if (eosr) {
-    // For cached buffers, we can't close the range until the cached buffer is returned.
-    // Close() is called from DiskIoMgr::ReturnBuffer().
-    if (!is_cached) scan_range->Close();
-  } else {
-    if (queue_full) {
-      reader->blocked_ranges_.Enqueue(scan_range);
-    } else {
-      reader->ScheduleScanRange(scan_range);
+  bool scan_range_done = eosr;
+  if (read_status.ok() && reader->state_ != RequestContext::Cancelled) {
+    DCHECK_EQ(reader->state_, RequestContext::Active);
+    // Read successfully - update the reader's scan ranges.  There are two cases here:
+    //  1. End of scan range
+    //  2. Middle of scan range
+    bool queue_full = scan_range->EnqueueBuffer(reader_lock, move(buffer));
+    if (!eosr) {
+      if (queue_full) {
+        reader->blocked_ranges_.Enqueue(scan_range);
+      } else {
+        reader->ScheduleScanRange(reader_lock, scan_range);
+      }
     }
+  } else {
+    // The scan range will be cancelled, either because we hit an error or because the
+    // request context was cancelled.  The buffer is not needed - we must free it.
+    reader->FreeBuffer(buffer.get());
+    reader->num_used_buffers_.Add(-1);
+    // Propagate the error or cancellation by cancelling the scan range.
+    scan_range->Cancel(read_status.ok() ? Status::CANCELLED : read_status);
+    scan_range_done = true;
+  }
+  if (scan_range_done) {
+    scan_range->Close();
+    --disk_state->num_remaining_ranges();
   }
-  state.DecrementRequestThread();
+  DCHECK(reader->Validate()) << endl << reader->DebugString();
+  disk_state->DecrementDiskThread(reader_lock, reader);
 }
 
 void DiskIoMgr::WorkLoop(DiskQueue* disk_queue) {
@@ -997,10 +664,13 @@ void DiskIoMgr::ReadRange(
   unique_ptr<BufferDescriptor> buffer_desc;
   if (range->external_buffer_tag_ == ScanRange::ExternalBufferTag::CLIENT_BUFFER) {
     buffer_desc = unique_ptr<BufferDescriptor>(new BufferDescriptor(this, reader, range,
-        range->client_buffer_.data, range->client_buffer_.len, nullptr));
+        range->client_buffer_.data, range->client_buffer_.len));
   } else {
+    DCHECK(range->external_buffer_tag_ == ScanRange::ExternalBufferTag::NO_BUFFER)
+        << "This code path does not handle other buffer types, i.e. HDFS cache"
+        << static_cast<int>(range->external_buffer_tag_);
     // Need to allocate a buffer to read into.
-    int64_t buffer_size = ::min(bytes_remaining, static_cast<int64_t>(max_buffer_size_));
+    int64_t buffer_size = min(bytes_remaining, max_buffer_size_);
     buffer_desc = TryAllocateNextBufferForRange(disk_queue, reader, range, buffer_size);
     if (buffer_desc == nullptr) return;
   }
@@ -1008,8 +678,8 @@ void DiskIoMgr::ReadRange(
 
   // No locks in this section.  Only working on local vars.  We don't want to hold a
   // lock across the read call.
-  buffer_desc->status_ = range->Open(detail::is_file_handle_caching_enabled());
-  if (buffer_desc->status_.ok()) {
+  Status read_status = range->Open(detail::is_file_handle_caching_enabled());
+  if (read_status.ok()) {
     // Update counters.
     if (reader->active_read_thread_counter_) {
       reader->active_read_thread_counter_->Add(1L);
@@ -1021,7 +691,7 @@ void DiskIoMgr::ReadRange(
     SCOPED_TIMER(&read_timer_);
     SCOPED_TIMER(reader->read_timer_);
 
-    buffer_desc->status_ = range->Read(buffer_desc->buffer_, buffer_desc->buffer_len_,
+    read_status = range->Read(buffer_desc->buffer_, buffer_desc->buffer_len_,
         &buffer_desc->len_, &buffer_desc->eosr_);
     buffer_desc->scan_range_offset_ = range->bytes_read_ - buffer_desc->len_;
 
@@ -1036,29 +706,25 @@ void DiskIoMgr::ReadRange(
   }
 
   // Finished read, update reader/disk based on the results
-  HandleReadFinished(disk_queue, reader, move(buffer_desc));
+  HandleReadFinished(disk_queue, reader, read_status, move(buffer_desc));
 }
 
 unique_ptr<BufferDescriptor> DiskIoMgr::TryAllocateNextBufferForRange(
     DiskQueue* disk_queue, RequestContext* reader, ScanRange* range,
     int64_t buffer_size) {
   DCHECK(reader->mem_tracker_ != nullptr);
+  // TODO: replace this with reservation check (if needed at all).
   bool enough_memory = reader->mem_tracker_->SpareCapacity() > LOW_MEMORY;
-  if (!enough_memory) {
-    // Low memory, GC all the buffers and try again.
-    GcIoBuffers();
-    enough_memory = reader->mem_tracker_->SpareCapacity() > LOW_MEMORY;
-  }
 
+  RequestContext::PerDiskState* disk_state = &reader->disk_states_[disk_queue->disk_id];
   if (!enough_memory) {
-    RequestContext::PerDiskState& state = reader->disk_states_[disk_queue->disk_id];
     unique_lock<mutex> reader_lock(reader->lock_);
 
     // Just grabbed the reader lock, check for cancellation.
     if (reader->state_ == RequestContext::Cancelled) {
       DCHECK(reader->Validate()) << endl << reader->DebugString();
-      state.DecrementRequestThreadAndCheckDone(reader);
-      range->Cancel(reader->status_);
+      disk_state->DecrementDiskThread(reader_lock, reader);
+      range->Cancel(Status::CANCELLED);
       DCHECK(reader->Validate()) << endl << reader->DebugString();
       return nullptr;
     }
@@ -1068,7 +734,7 @@ unique_ptr<BufferDescriptor> DiskIoMgr::TryAllocateNextBufferForRange(
       // (it already has one queued). Skip this range and pick it up later.
       range->blocked_on_queue_ = true;
       reader->blocked_ranges_.Enqueue(range);
-      state.DecrementRequestThread();
+      disk_state->DecrementDiskThread(reader_lock, reader);
       return nullptr;
     } else {
       // We need to get a buffer anyway since there are none queued. The query
@@ -1076,7 +742,18 @@ unique_ptr<BufferDescriptor> DiskIoMgr::TryAllocateNextBufferForRange(
       // now.
     }
   }
-  unique_ptr<BufferDescriptor> buffer_desc = GetFreeBuffer(reader, range, buffer_size);
+  unique_ptr<BufferDescriptor> buffer_desc;
+  Status status = reader->AllocBuffer(range, buffer_size, &buffer_desc);
+  if (!status.ok()) {
+    // Hit memory limit - cancel range.
+    range->Cancel(status);
+    {
+      unique_lock<mutex> reader_lock(reader->lock_);
+      disk_state->DecrementDiskThread(reader_lock, reader);
+      DCHECK(reader->Validate()) << endl << reader->DebugString();
+    }
+    return nullptr;
+  }
   DCHECK(buffer_desc != nullptr);
   return buffer_desc;
 }
@@ -1139,23 +816,10 @@ Status DiskIoMgr::WriteRangeHelper(FILE* file_handle, WriteRange* write_range) {
   return Status::OK();
 }
 
-int DiskIoMgr::free_buffers_idx(int64_t buffer_size) {
-  int64_t buffer_size_scaled = BitUtil::Ceil(buffer_size, min_buffer_size_);
-  int idx = BitUtil::Log2Ceiling64(buffer_size_scaled);
-  DCHECK_GE(idx, 0);
-  DCHECK_LT(idx, free_buffers_.size());
-  return idx;
-}
-
 Status DiskIoMgr::AddWriteRange(RequestContext* writer, WriteRange* write_range) {
   unique_lock<mutex> writer_lock(writer->lock_);
-
-  if (writer->state_ == RequestContext::Cancelled) {
-    DCHECK(!writer->status_.ok());
-    return writer->status_;
-  }
-
-  writer->AddRequestRange(write_range, false);
+  if (writer->state_ == RequestContext::Cancelled) return Status::CANCELLED;
+  writer->AddRequestRange(writer_lock, write_range, false);
   return Status::OK();
 }
 

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/io/disk-io-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr.h b/be/src/runtime/io/disk-io-mgr.h
index 4fa078b..20f5d44 100644
--- a/be/src/runtime/io/disk-io-mgr.h
+++ b/be/src/runtime/io/disk-io-mgr.h
@@ -144,8 +144,8 @@ namespace io {
 ///
 /// As a caller reads from a scan range, these buffers are wrapped in BufferDescriptors
 /// and returned to the caller. The caller must always call ReturnBuffer() on the buffer
-/// descriptor to allow recycling of the associated buffer (if there is an
-/// IoMgr-allocated or HDFS cached buffer).
+/// descriptor to allow freeing of the associated buffer (if there is an IoMgr-allocated
+/// or HDFS cached buffer).
 ///
 /// Caching support:
 /// Scan ranges contain metadata on whether or not it is cached on the DN. In that
@@ -207,7 +207,7 @@ class DiskIoMgr : public CacheLineAligned {
   ///  - min_buffer_size: minimum io buffer size (in bytes)
   ///  - max_buffer_size: maximum io buffer size (in bytes). Also the max read size.
   DiskIoMgr(int num_disks, int threads_per_rotational_disk,
-      int threads_per_solid_state_disk, int min_buffer_size, int max_buffer_size);
+      int threads_per_solid_state_disk, int64_t min_buffer_size, int64_t max_buffer_size);
 
   /// Create DiskIoMgr with default configs.
   DiskIoMgr();
@@ -217,7 +217,8 @@ class DiskIoMgr : public CacheLineAligned {
   ~DiskIoMgr();
 
   /// Initialize the IoMgr. Must be called once before any of the other APIs.
-  Status Init(MemTracker* process_mem_tracker) WARN_UNUSED_RESULT;
+  Status Init() WARN_UNUSED_RESULT;
+
 
   /// Allocates tracking structure for a request context.
   /// Register a new request context and return it to the caller. The caller must call
@@ -236,12 +237,6 @@ class DiskIoMgr : public CacheLineAligned {
   /// up.
   void UnregisterContext(RequestContext* context);
 
-  /// This function cancels the context asychronously. All outstanding requests
-  /// are aborted and tracking structures cleaned up. This does not need to be
-  /// called if the context finishes normally.
-  /// This will also fail any outstanding GetNext()/Read requests.
-  void CancelContext(RequestContext* context);
-
   /// Adds the scan ranges to the queues. This call is non-blocking. The caller must
   /// not deallocate the scan range pointers before UnregisterContext().
   /// If schedule_immediately, the ranges are immediately put on the read queue
@@ -266,52 +261,14 @@ class DiskIoMgr : public CacheLineAligned {
   /// This call is blocking.
   Status GetNextRange(RequestContext* reader, ScanRange** range) WARN_UNUSED_RESULT;
 
-  /// Reads the range and returns the result in buffer.
-  /// This behaves like the typical synchronous read() api, blocking until the data
-  /// is read. This can be called while there are outstanding ScanRanges and is
-  /// thread safe. Multiple threads can be calling Read() per reader at a time.
-  /// range *cannot* have already been added via AddScanRanges.
-  /// This can only be used if the scan range fits in a single IO buffer (i.e. is smaller
-  /// than max_read_buffer_size()) or if reading into a client-provided buffer.
-  Status Read(RequestContext* reader, ScanRange* range,
-      std::unique_ptr<BufferDescriptor>* buffer) WARN_UNUSED_RESULT;
-
-  /// Returns the buffer to the IoMgr. This must be called for every buffer
-  /// returned by GetNext()/Read() that did not return an error. This is non-blocking.
-  /// After calling this, the buffer descriptor is invalid and cannot be accessed.
-  void ReturnBuffer(std::unique_ptr<BufferDescriptor> buffer);
-
   /// Determine which disk queue this file should be assigned to.  Returns an index into
   /// disk_queues_.  The disk_id is the volume ID for the local disk that holds the
   /// files, or -1 if unknown.  Flag expected_local is true iff this impalad is
   /// co-located with the datanode for this file.
   int AssignQueue(const char* file, int disk_id, bool expected_local);
 
-  /// TODO: The functions below can be moved to RequestContext.
-  /// Returns the current status of the context.
-  Status context_status(RequestContext* context) const WARN_UNUSED_RESULT;
-
-  void set_bytes_read_counter(RequestContext*, RuntimeProfile::Counter*);
-  void set_read_timer(RequestContext*, RuntimeProfile::Counter*);
-  void set_active_read_thread_counter(RequestContext*, RuntimeProfile::Counter*);
-  void set_disks_access_bitmap(RequestContext*, RuntimeProfile::Counter*);
-
-  int64_t queue_size(RequestContext* reader) const;
-  int64_t bytes_read_local(RequestContext* reader) const;
-  int64_t bytes_read_short_circuit(RequestContext* reader) const;
-  int64_t bytes_read_dn_cache(RequestContext* reader) const;
-  int num_remote_ranges(RequestContext* reader) const;
-  int64_t unexpected_remote_bytes(RequestContext* reader) const;
-  int cached_file_handles_hit_count(RequestContext* reader) const;
-  int cached_file_handles_miss_count(RequestContext* reader) const;
-
-  /// Returns the read throughput across all readers.
-  /// TODO: should this be a sliding window?  This should report metrics for the
-  /// last minute, hour and since the beginning.
-  int64_t GetReadThroughput();
-
   /// Returns the maximum read buffer size
-  int max_read_buffer_size() const { return max_buffer_size_; }
+  int64_t max_read_buffer_size() const { return max_buffer_size_; }
 
   /// Returns the total number of disk queues (both local and remote).
   int num_total_disks() const { return disk_queues_.size(); }
@@ -361,10 +318,6 @@ class DiskIoMgr : public CacheLineAligned {
   Status ReopenCachedHdfsFileHandle(const hdfsFS& fs, std::string* fname, int64_t mtime,
       CachedHdfsFileHandle** fid);
 
-  /// Garbage collect unused I/O buffers up to 'bytes_to_free', or all the buffers if
-  /// 'bytes_to_free' is -1.
-  void GcIoBuffers(int64_t bytes_to_free = -1);
-
   /// The maximum number of ready buffers that can be queued in a scan range. Having two
   /// queued buffers (plus the buffer that is returned to the client) gives good
   /// performance in most scenarios:
@@ -403,14 +356,6 @@ class DiskIoMgr : public CacheLineAligned {
   friend class DiskIoMgrTest_Buffers_Test;
   friend class DiskIoMgrTest_VerifyNumThreadsParameter_Test;
 
-  /// Memory tracker for unused I/O buffers owned by DiskIoMgr.
-  boost::scoped_ptr<MemTracker> free_buffer_mem_tracker_;
-
-  /// Memory tracker for I/O buffers where the RequestContext has no MemTracker.
-  /// TODO: once IMPALA-3200 is fixed, there should be no more cases where readers don't
-  /// provide a MemTracker.
-  boost::scoped_ptr<MemTracker> unowned_buffer_mem_tracker_;
-
   /// Number of worker(read) threads per rotational disk. Also the max depth of queued
   /// work to the disk.
   const int num_io_threads_per_rotational_disk_;
@@ -420,10 +365,10 @@ class DiskIoMgr : public CacheLineAligned {
   const int num_io_threads_per_solid_state_disk_;
 
   /// Maximum read size. This is also the maximum size of each allocated buffer.
-  const int max_buffer_size_;
+  const int64_t max_buffer_size_;
 
   /// The minimum size of each read buffer.
-  const int min_buffer_size_;
+  const int64_t min_buffer_size_;
 
   /// Thread group containing all the worker threads.
   ThreadGroup disk_thread_group_;
@@ -441,28 +386,6 @@ class DiskIoMgr : public CacheLineAligned {
   /// Total time spent in hdfs reading
   RuntimeProfile::Counter read_timer_;
 
-  /// Protects free_buffers_
-  boost::mutex free_buffers_lock_;
-
-  /// Free buffers that can be handed out to clients. There is one list for each buffer
-  /// size, indexed by the Log2 of the buffer size in units of min_buffer_size_. The
-  /// maximum buffer size is max_buffer_size_, so the maximum index is
-  /// Log2(max_buffer_size_ / min_buffer_size_).
-  //
-  /// E.g. if min_buffer_size_ = 1024 bytes:
-  ///  free_buffers_[0]  => list of free buffers with size 1024 B
-  ///  free_buffers_[1]  => list of free buffers with size 2048 B
-  ///  free_buffers_[10] => list of free buffers with size 1 MB
-  ///  free_buffers_[13] => list of free buffers with size 8 MB
-  ///  free_buffers_[n]  => list of free buffers with size 2^n * 1024 B
-  std::vector<std::deque<uint8_t*>> free_buffers_;
-
-  /// Total number of allocated buffers, used for debugging.
-  AtomicInt32 num_allocated_buffers_;
-
-  /// Total number of buffers in readers
-  AtomicInt32 num_buffers_in_readers_;
-
   /// Per disk queues. This is static and created once at Init() time.  One queue is
   /// allocated for each local disk on the system and for each remote filesystem type.
   /// It is indexed by disk id.
@@ -478,23 +401,6 @@ class DiskIoMgr : public CacheLineAligned {
   // handles are closed.
   FileHandleCache file_handle_cache_;
 
-  /// Returns the index into free_buffers_ for a given buffer size
-  int free_buffers_idx(int64_t buffer_size);
-
-  /// Returns a buffer to read into with size between 'buffer_size' and
-  /// 'max_buffer_size_', If there is an appropriately-sized free buffer in the
-  /// 'free_buffers_', that is returned, otherwise a new one is allocated.
-  /// The returned *buffer_size must be between 0 and 'max_buffer_size_'.
-  /// The buffer memory is tracked against reader's mem tracker, or
-  /// 'unowned_buffer_mem_tracker_' if the reader does not have one.
-  std::unique_ptr<BufferDescriptor> GetFreeBuffer(
-      RequestContext* reader, ScanRange* range, int64_t buffer_size);
-
-  /// Disassociates the desc->buffer_ memory from 'desc' (which cannot be nullptr), either
-  /// freeing it or returning it to 'free_buffers_'. Memory tracking is updated to
-  /// reflect the transfer of ownership from desc->mem_tracker_ to the disk I/O mgr.
-  void FreeBufferMemory(BufferDescriptor* desc);
-
   /// Disk worker thread loop. This function retrieves the next range to process on
   /// the disk queue and invokes ReadRange() or Write() depending on the type of Range().
   /// There can be multiple threads per disk running this loop.
@@ -508,10 +414,12 @@ class DiskIoMgr : public CacheLineAligned {
   bool GetNextRequestRange(DiskQueue* disk_queue, RequestRange** range,
       RequestContext** request_context);
 
-  /// Updates disk queue and reader state after a read is complete. The read result
-  /// is captured in the buffer descriptor.
+  /// Updates disk queue and reader state after a read is complete. If the read
+  /// was successful, 'read_status' is ok and 'buffer' contains the result of the
+  /// read. If the read failed with an error, 'read_status' contains the error and
+  /// 'buffer' has the buffer that was meant to hold the result of the read.
   void HandleReadFinished(DiskQueue* disk_queue, RequestContext* reader,
-      std::unique_ptr<BufferDescriptor> buffer);
+      Status read_status, std::unique_ptr<BufferDescriptor> buffer);
 
   /// Invokes write_range->callback_  after the range has been written and
   /// updates per-disk state and handle state. The status of the write OK/RUNTIME_ERROR
@@ -534,13 +442,15 @@ class DiskIoMgr : public CacheLineAligned {
   /// Does not open or close the file that is written.
   Status WriteRangeHelper(FILE* file_handle, WriteRange* write_range) WARN_UNUSED_RESULT;
 
-  /// Reads the specified scan range and calls HandleReadFinished when done.
+  /// Reads the specified scan range and calls HandleReadFinished() when done.
   void ReadRange(DiskQueue* disk_queue, RequestContext* reader, ScanRange* range);
 
   /// Try to allocate the next buffer for the scan range, returning the new buffer
   /// if successful. If 'reader' is cancelled, cancels the range and returns nullptr.
   /// If there is memory pressure and buffers are already queued, adds the range
-  /// to the blocked ranges and returns nullptr.
+  /// to the blocked ranges and returns nullptr. If buffers are not queued and no more
+  /// buffers can be allocated, cancels the range with a MEM_LIMIT_EXCEEDED error and
+  /// returns nullptr.
   std::unique_ptr<BufferDescriptor> TryAllocateNextBufferForRange(DiskQueue* disk_queue,
       RequestContext* reader, ScanRange* range, int64_t buffer_size);
 };

http://git-wip-us.apache.org/repos/asf/impala/blob/3b3bf871/be/src/runtime/io/request-context.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-context.cc b/be/src/runtime/io/request-context.cc
index 287f53a..031b976 100644
--- a/be/src/runtime/io/request-context.cc
+++ b/be/src/runtime/io/request-context.cc
@@ -22,32 +22,100 @@
 using namespace impala;
 using namespace impala::io;
 
-void RequestContext::Cancel(const Status& status) {
-  DCHECK(!status.ok());
+BufferDescriptor::BufferDescriptor(DiskIoMgr* io_mgr,
+    RequestContext* reader, ScanRange* scan_range, uint8_t* buffer,
+    int64_t buffer_len)
+  : io_mgr_(io_mgr),
+    reader_(reader),
+    scan_range_(scan_range),
+    buffer_(buffer),
+    buffer_len_(buffer_len) {
+  DCHECK(io_mgr != nullptr);
+  DCHECK(scan_range != nullptr);
+  DCHECK(buffer != nullptr);
+  DCHECK_GE(buffer_len, 0);
+}
+
+Status RequestContext::AllocBuffer(ScanRange* range, int64_t buffer_size,
+    unique_ptr<BufferDescriptor>* buffer_desc) {
+  DCHECK(range->external_buffer_tag_ == ScanRange::ExternalBufferTag::NO_BUFFER)
+      << static_cast<int>(range->external_buffer_tag_);
+  DCHECK_LE(buffer_size, parent_->max_buffer_size_);
+  DCHECK_GT(buffer_size, 0);
+  buffer_size = BitUtil::RoundUpToPowerOfTwo(
+      max(parent_->min_buffer_size_, min(parent_->max_buffer_size_, buffer_size)));
+
+  DCHECK(mem_tracker_ != nullptr);
+  if (!mem_tracker_->TryConsume(buffer_size)) {
+    return mem_tracker_->MemLimitExceeded(nullptr, "disk I/O buffer", buffer_size);
+  }
+
+  uint8_t* buffer = reinterpret_cast<uint8_t*>(malloc(buffer_size));
+  if (buffer == nullptr) {
+    mem_tracker_->Release(buffer_size);
+    return Status(TErrorCode::INTERNAL_ERROR,
+        Substitute("Could not malloc buffer of $0 bytes"));
+  }
+  buffer_desc->reset(new BufferDescriptor(parent_, this, range, buffer, buffer_size));
+  return Status::OK();
+}
+
+void RequestContext::FreeBuffer(BufferDescriptor* buffer) {
+  DCHECK(buffer->buffer_ != nullptr);
+  if (!buffer->is_cached() && !buffer->is_client_buffer()) {
+    // Only buffers that were not allocated by DiskIoMgr need to have memory freed.
+    free(buffer->buffer_);
+    mem_tracker_->Release(buffer->buffer_len_);
+  }
+  buffer->buffer_ = nullptr;
+}
 
+// Cancellation of a RequestContext requires coordination from multiple threads that may
+// hold references to the context:
+//  1. Disk threads that are currently processing a range for this context.
+//  2. Caller threads that are waiting in GetNext().
+//
+// Each thread that currently has a reference to the request context must notice the
+// cancel, cancel any pending operations involving the context and remove the contxt from
+// tracking structures. Once no more operations are pending on the context and no more
+// I/O mgr threads hold references to the context, the context can be marked inactive
+// (see CancelAndMarkInactive()), after which the owner of the context object can free
+// it.
+//
+// The steps are:
+// 1. Cancel() will immediately set the context in the Cancelled state. This prevents any
+// other thread from adding more ready buffers to the context (they all take a lock and
+// check the state before doing so), or any write ranges to the context.
+// 2. Cancel() will call Cancel() on each ScanRange that is not yet complete, unblocking
+// any threads in GetNext(). If there was no prior error for a scan range, any reads from
+// that scan range will return a CANCELLED Status. Cancel() also invokes callbacks for
+// all WriteRanges with a CANCELLED Status.
+// 3. Disk threads notice the context is cancelled either when picking the next context
+// to process or when they try to enqueue a ready buffer. Upon noticing the cancelled
+// state, removes the context from the disk queue. The last thread per disk then calls
+// DecrementDiskRefCount(). After the last disk thread has called DecrementDiskRefCount(),
+// cancellation is done and it is safe to unregister the context.
+void RequestContext::Cancel() {
   // Callbacks are collected in this vector and invoked while no lock is held.
   vector<WriteRange::WriteDoneCallback> write_callbacks;
   {
-    lock_guard<mutex> lock(lock_);
+    unique_lock<mutex> lock(lock_);
     DCHECK(Validate()) << endl << DebugString();
 
     // Already being cancelled
     if (state_ == RequestContext::Cancelled) return;
 
-    DCHECK(status_.ok());
-    status_ = status;
-
     // The reader will be put into a cancelled state until call cleanup is complete.
     state_ = RequestContext::Cancelled;
 
     // Cancel all scan ranges for this reader. Each range could be one one of
     // four queues.
     for (int i = 0; i < disk_states_.size(); ++i) {
-      RequestContext::PerDiskState& state = disk_states_[i];
-      RequestRange* range = NULL;
-      while ((range = state.in_flight_ranges()->Dequeue()) != NULL) {
+      PerDiskState& state = disk_states_[i];
+      RequestRange* range = nullptr;
+      while ((range = state.in_flight_ranges()->Dequeue()) != nullptr) {
         if (range->request_type() == RequestType::READ) {
-          static_cast<ScanRange*>(range)->Cancel(status);
+          static_cast<ScanRange*>(range)->Cancel(Status::CANCELLED);
         } else {
           DCHECK(range->request_type() == RequestType::WRITE);
           write_callbacks.push_back(static_cast<WriteRange*>(range)->callback_);
@@ -55,36 +123,36 @@ void RequestContext::Cancel(const Status& status) {
       }
 
       ScanRange* scan_range;
-      while ((scan_range = state.unstarted_scan_ranges()->Dequeue()) != NULL) {
-        scan_range->Cancel(status);
+      while ((scan_range = state.unstarted_scan_ranges()->Dequeue()) != nullptr) {
+        scan_range->Cancel(Status::CANCELLED);
       }
       WriteRange* write_range;
-      while ((write_range = state.unstarted_write_ranges()->Dequeue()) != NULL) {
+      while ((write_range = state.unstarted_write_ranges()->Dequeue()) != nullptr) {
         write_callbacks.push_back(write_range->callback_);
       }
     }
 
-    ScanRange* range = NULL;
-    while ((range = ready_to_start_ranges_.Dequeue()) != NULL) {
-      range->Cancel(status);
+    ScanRange* range = nullptr;
+    while ((range = ready_to_start_ranges_.Dequeue()) != nullptr) {
+      range->Cancel(Status::CANCELLED);
     }
-    while ((range = blocked_ranges_.Dequeue()) != NULL) {
-      range->Cancel(status);
+    while ((range = blocked_ranges_.Dequeue()) != nullptr) {
+      range->Cancel(Status::CANCELLED);
     }
-    while ((range = cached_ranges_.Dequeue()) != NULL) {
-      range->Cancel(status);
+    while ((range = cached_ranges_.Dequeue()) != nullptr) {
+      range->Cancel(Status::CANCELLED);
     }
 
-    // Schedule reader on all disks. The disks will notice it is cancelled and do any
-    // required cleanup
+    // Ensure that the reader is scheduled on all disks (it may already be scheduled on
+    // some). The disk threads will notice that the context is cancelled and do any
+    // required cleanup for the disk state.
     for (int i = 0; i < disk_states_.size(); ++i) {
-      RequestContext::PerDiskState& state = disk_states_[i];
-      state.ScheduleContext(this, i);
+      disk_states_[i].ScheduleContext(lock, this, i);
     }
   }
 
   for (const WriteRange::WriteDoneCallback& write_callback: write_callbacks) {
-    write_callback(status_);
+    write_callback(Status::CANCELLED);
   }
 
   // Signal reader and unblock the GetNext/Read thread.  That read will fail with
@@ -93,7 +161,7 @@ void RequestContext::Cancel(const Status& status) {
 }
 
 void RequestContext::CancelAndMarkInactive() {
-  Cancel(Status::CANCELLED);
+  Cancel();
 
   boost::unique_lock<boost::mutex> l(lock_);
   DCHECK_NE(state_, Inactive);
@@ -105,14 +173,22 @@ void RequestContext::CancelAndMarkInactive() {
   // Validate that no buffers were leaked from this context.
   DCHECK_EQ(num_buffers_in_reader_.Load(), 0) << endl << DebugString();
   DCHECK_EQ(num_used_buffers_.Load(), 0) << endl << DebugString();
+
+  // Validate that no threads are active and the context is not queued.
+  for (const PerDiskState& disk_state : disk_states_) {
+    DCHECK_EQ(0, disk_state.in_flight_ranges()->size()) << endl << DebugString();
+    DCHECK_EQ(0, disk_state.unstarted_scan_ranges()->size()) << endl << DebugString();
+    DCHECK_EQ(0, disk_state.num_threads_in_op()) << endl << DebugString();
+    DCHECK(!disk_state.is_on_queue()) << endl << DebugString();
+  }
   DCHECK(Validate()) << endl << DebugString();
   state_ = Inactive;
 }
 
-void RequestContext::AddRequestRange(
+void RequestContext::AddRequestRange(const unique_lock<mutex>& lock,
     RequestRange* range, bool schedule_immediately) {
-  // DCHECK(lock_.is_locked()); // TODO: boost should have this API
-  RequestContext::PerDiskState& state = disk_states_[range->disk_id()];
+  DCHECK(lock.mutex() == &lock_ && lock.owns_lock());
+  PerDiskState& state = disk_states_[range->disk_id()];
   if (state.done()) {
     DCHECK_EQ(state.num_remaining_ranges(), 0);
     state.set_done(false);
@@ -123,7 +199,7 @@ void RequestContext::AddRequestRange(
   if (range->request_type() == RequestType::READ) {
     ScanRange* scan_range = static_cast<ScanRange*>(range);
     if (schedule_immediately) {
-      ScheduleScanRange(scan_range);
+      ScheduleScanRange(lock, scan_range);
     } else {
       state.unstarted_scan_ranges()->Enqueue(scan_range);
       num_unstarted_scan_ranges_.Add(1);
@@ -143,7 +219,7 @@ void RequestContext::AddRequestRange(
     schedule_context = true;
   }
 
-  if (schedule_context) state.ScheduleContext(this, range->disk_id());
+  if (schedule_context) state.ScheduleContext(lock, this, range->disk_id());
   ++state.num_remaining_ranges();
 }
 
@@ -159,8 +235,7 @@ string RequestContext::DebugString() const {
   if (state_ == RequestContext::Cancelled) ss << "Cancelled";
   if (state_ == RequestContext::Active) ss << "Active";
   if (state_ != RequestContext::Inactive) {
-    ss << " status_=" << (status_.ok() ? "OK" : status_.GetDetail())
-       << " #ready_buffers=" << num_ready_buffers_.Load()
+    ss << " #ready_buffers=" << num_ready_buffers_.Load()
        << " #used_buffers=" << num_used_buffers_.Load()
        << " #num_buffers_in_reader=" << num_buffers_in_reader_.Load()
        << " #finished_scan_ranges=" << num_finished_ranges_.Load()
@@ -284,10 +359,11 @@ bool RequestContext::Validate() const {
   return true;
 }
 
-void RequestContext::PerDiskState::ScheduleContext(
+void RequestContext::PerDiskState::ScheduleContext(const unique_lock<mutex>& context_lock,
     RequestContext* context, int disk_id) {
-  if (!is_on_queue_ && !done_) {
-    is_on_queue_ = true;
+  DCHECK(context_lock.mutex() == &context->lock_ && context_lock.owns_lock());
+  if (is_on_queue_.Load() == 0 && !done_) {
+    is_on_queue_.Store(1);
     context->parent_->disk_queues_[disk_id]->EnqueueContext(context);
   }
 }


[04/15] impala git commit: IMPALA-4835: Part 3: switch I/O buffers to buffer pool

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
index 7735f98..87d1806 100644
--- a/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/HdfsScanNode.java
@@ -59,6 +59,7 @@ import org.apache.impala.common.ImpalaException;
 import org.apache.impala.common.ImpalaRuntimeException;
 import org.apache.impala.common.InternalException;
 import org.apache.impala.common.NotImplementedException;
+import org.apache.impala.common.Pair;
 import org.apache.impala.common.PrintUtils;
 import org.apache.impala.common.RuntimeEnv;
 import org.apache.impala.fb.FbFileBlock;
@@ -76,6 +77,7 @@ import org.apache.impala.thrift.TScanRange;
 import org.apache.impala.thrift.TScanRangeLocation;
 import org.apache.impala.thrift.TScanRangeLocationList;
 import org.apache.impala.thrift.TTableStats;
+import org.apache.impala.util.BitUtil;
 import org.apache.impala.util.MembershipSnapshot;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -143,7 +145,7 @@ public class HdfsScanNode extends ScanNode {
   // derived experimentally: running metadata-only Parquet count(*) scans on TPC-H
   // lineitem and TPC-DS store_sales of different sizes resulted in memory consumption
   // between 128kb and 1.1mb.
-  private final static long MIN_MEMORY_ESTIMATE = 1 * 1024 * 1024;
+  private static final long MIN_MEMORY_ESTIMATE = 1L * 1024L * 1024L;
 
   private final HdfsTable tbl_;
 
@@ -166,6 +168,18 @@ public class HdfsScanNode extends ScanNode {
   private long totalFiles_ = 0;
   private long totalBytes_ = 0;
 
+  // File formats scanned. Set in computeScanRangeLocations().
+  private Set<HdfsFileFormat> fileFormats_;
+
+  // Number of bytes in the largest scan range (i.e. hdfs split). Set in
+  // computeScanRangeLocations().
+  private long maxScanRangeBytes_ = 0;
+
+  // The ideal reservation to process a single scan range (i.e. hdfs split), >= the
+  // minimum reservation. Generally provides enough memory to overlap CPU and I/O and
+  // maximize throughput. Set in computeResourceProfile().
+  private long idealScanRangeReservation_ = -1;
+
   // Input cardinality based on the partition row counts or extrapolation. -1 if invalid.
   // Both values can be valid to report them in the explain plan, but only one of them is
   // used for determining the scan cardinality.
@@ -329,25 +343,25 @@ public class HdfsScanNode extends ScanNode {
     computeDictionaryFilterConjuncts(analyzer);
 
     // compute scan range locations with optional sampling
-    Set<HdfsFileFormat> fileFormats = computeScanRangeLocations(analyzer);
+    computeScanRangeLocations(analyzer);
 
     // Determine backend scan node implementation to use. The optimized MT implementation
     // is currently only supported for Parquet.
     if (analyzer.getQueryOptions().isSetMt_dop() &&
         analyzer.getQueryOptions().mt_dop > 0 &&
-        fileFormats.size() == 1 &&
-        (fileFormats.contains(HdfsFileFormat.PARQUET)
-          || fileFormats.contains(HdfsFileFormat.TEXT))) {
+        fileFormats_.size() == 1 &&
+        (fileFormats_.contains(HdfsFileFormat.PARQUET)
+          || fileFormats_.contains(HdfsFileFormat.TEXT))) {
       useMtScanNode_ = true;
     } else {
       useMtScanNode_ = false;
     }
 
-    if (fileFormats.contains(HdfsFileFormat.PARQUET)) {
+    if (fileFormats_.contains(HdfsFileFormat.PARQUET)) {
       computeMinMaxTupleAndConjuncts(analyzer);
     }
 
-    if (canApplyParquetCountStarOptimization(analyzer, fileFormats)) {
+    if (canApplyParquetCountStarOptimization(analyzer, fileFormats_)) {
       Preconditions.checkState(desc_.getPath().destTable() != null);
       Preconditions.checkState(collectionConjuncts_.isEmpty());
       countStarSlot_ = applyParquetCountStartOptimization(analyzer);
@@ -460,7 +474,7 @@ public class HdfsScanNode extends ScanNode {
     // This node is a table scan, so this must be a scanning slot.
     Preconditions.checkState(slotRef.getDesc().isScanSlot());
     // Skip the slot ref if it refers to an array's "pos" field.
-    if (slotRef.isArrayPosRef()) return;
+    if (slotRef.getDesc().isArrayPosRef()) return;
 
     Expr constExpr = binaryPred.getChild(1);
     // Only constant exprs can be evaluated against parquet::Statistics. This includes
@@ -488,7 +502,7 @@ public class HdfsScanNode extends ScanNode {
     // This node is a table scan, so this must be a scanning slot.
     Preconditions.checkState(slotRef.getDesc().isScanSlot());
     // Skip the slot ref if it refers to an array's "pos" field.
-    if (slotRef.isArrayPosRef()) return;
+    if (slotRef.getDesc().isArrayPosRef()) return;
     if (inPred.isNotIn()) return;
 
     ArrayList<Expr> children = inPred.getChildren();
@@ -695,13 +709,15 @@ public class HdfsScanNode extends ScanNode {
   }
 
   /**
-   * Computes scan ranges (hdfs splits) plus their storage locations, including volume
-   * ids, based on the given maximum number of bytes each scan range should scan.
+   * Computes scan ranges (i.e. hdfs splits) plus their storage locations, including
+   * volume ids, based on the given maximum number of bytes each scan range should scan.
    * If 'sampleParams_' is not null, generates a sample and computes the scan ranges
    * based on the sample.
-   * Returns the set of file formats being scanned.
+   *
+   * Initializes members with information about files and scan ranges, e.g. totalFiles_,
+   * fileFormats_, etc.
    */
-  private Set<HdfsFileFormat> computeScanRangeLocations(Analyzer analyzer)
+  private void computeScanRangeLocations(Analyzer analyzer)
       throws ImpalaRuntimeException {
     Map<Long, List<FileDescriptor>> sampledFiles = null;
     if (sampleParams_ != null) {
@@ -724,7 +740,8 @@ public class HdfsScanNode extends ScanNode {
     numPartitions_ = (sampledFiles != null) ? sampledFiles.size() : partitions_.size();
     totalFiles_ = 0;
     totalBytes_ = 0;
-    Set<HdfsFileFormat> fileFormats = Sets.newHashSet();
+    maxScanRangeBytes_ = 0;
+    fileFormats_ = Sets.newHashSet();
     for (HdfsPartition partition: partitions_) {
       List<FileDescriptor> fileDescs = partition.getFileDescriptors();
       if (sampledFiles != null) {
@@ -734,7 +751,7 @@ public class HdfsScanNode extends ScanNode {
       }
 
       analyzer.getDescTbl().addReferencedPartition(tbl_, partition.getId());
-      fileFormats.add(partition.getFileFormat());
+      fileFormats_.add(partition.getFileFormat());
       Preconditions.checkState(partition.getId() >= 0);
       // Missing disk id accounting is only done for file systems that support the notion
       // of disk/storage ids.
@@ -797,6 +814,7 @@ public class HdfsScanNode extends ScanNode {
             scanRangeLocations.scan_range = scanRange;
             scanRangeLocations.locations = locations;
             scanRanges_.add(scanRangeLocations);
+            maxScanRangeBytes_ = Math.max(maxScanRangeBytes_, currentLength);
             remainingLength -= currentLength;
             currentOffset += currentLength;
           }
@@ -811,7 +829,58 @@ public class HdfsScanNode extends ScanNode {
       }
       if (partitionMissingDiskIds) ++numPartitionsNoDiskIds_;
     }
-    return fileFormats;
+  }
+
+  /**
+   * Compute the number of columns that are read from the file, as opposed to
+   * materialised based on metadata. If there are nested collections, counts the
+   * number of leaf scalar slots per collection. This matches Parquet's "shredded"
+   * approach to nested collections, where each nested field is stored as a separate
+   * column. We may need to adjust this logic for non-shredded columnar formats if added.
+   */
+  private int computeNumColumnsReadFromFile() {
+    HdfsTable table = (HdfsTable) desc_.getTable();
+    int numColumns = 0;
+    boolean havePosSlot = false;
+    for (SlotDescriptor slot: desc_.getSlots()) {
+      if (!slot.isMaterialized() || slot == countStarSlot_) continue;
+      if (slot.getColumn() == null ||
+          slot.getColumn().getPosition() >= table.getNumClusteringCols()) {
+        if (slot.isArrayPosRef()) {
+          // Position virtual slots can be materialized by piggybacking on another slot.
+          havePosSlot = true;
+        } else if (slot.getType().isScalarType()) {
+          ++numColumns;
+        } else {
+          numColumns += computeNumColumnsReadForCollection(slot);
+        }
+      }
+    }
+    // Must scan something to materialize a position slot.
+    if (havePosSlot) numColumns = Math.max(numColumns, 1);
+    return numColumns;
+  }
+
+  /**
+   * Compute the number of columns read from disk for materialized scalar slots in
+   * the provided tuple.
+   */
+  private int computeNumColumnsReadForCollection(SlotDescriptor collectionSlot) {
+    Preconditions.checkState(collectionSlot.getType().isCollectionType());
+    int numColumns = 0;
+    for (SlotDescriptor nestedSlot: collectionSlot.getItemTupleDesc().getSlots()) {
+      // Position virtual slots can be materialized by piggybacking on another slot.
+      if (!nestedSlot.isMaterialized() || nestedSlot.isArrayPosRef()) continue;
+      if (nestedSlot.getType().isScalarType()) {
+        ++numColumns;
+      } else {
+        numColumns += computeNumColumnsReadForCollection(nestedSlot);
+      }
+    }
+    // Need to scan at least one column to materialize the pos virtual slot and/or
+    // determine the size of the nested array.
+    numColumns = Math.max(numColumns, 1);
+    return numColumns;
   }
 
   /**
@@ -1006,6 +1075,8 @@ public class HdfsScanNode extends ScanNode {
     }
     msg.hdfs_scan_node.setRandom_replica(randomReplica_);
     msg.node_type = TPlanNodeType.HDFS_SCAN_NODE;
+    Preconditions.checkState(idealScanRangeReservation_ >= 0, idealScanRangeReservation_);
+    msg.hdfs_scan_node.setIdeal_scan_range_reservation(idealScanRangeReservation_);
     if (!collectionConjuncts_.isEmpty()) {
       Map<Integer, List<TExpr>> tcollectionConjuncts = Maps.newLinkedHashMap();
       for (Map.Entry<TupleDescriptor, List<Expr>> entry:
@@ -1184,25 +1255,20 @@ public class HdfsScanNode extends ScanNode {
     Preconditions.checkNotNull(scanRanges_, "Cost estimation requires scan ranges.");
     if (scanRanges_.isEmpty()) {
       nodeResourceProfile_ = ResourceProfile.noReservation(0);
+      idealScanRangeReservation_ = 0;
       return;
     }
     Preconditions.checkState(0 < numNodes_ && numNodes_ <= scanRanges_.size());
     Preconditions.checkNotNull(desc_);
     Preconditions.checkNotNull(desc_.getTable() instanceof HdfsTable);
     HdfsTable table = (HdfsTable) desc_.getTable();
+    int numColumnsReadFromFile = computeNumColumnsReadFromFile();
     int perHostScanRanges;
     if (table.getMajorityFormat() == HdfsFileFormat.PARQUET) {
       // For the purpose of this estimation, the number of per-host scan ranges for
       // Parquet files are equal to the number of columns read from the file. I.e.
       // excluding partition columns and columns that are populated from file metadata.
-      perHostScanRanges = 0;
-      for (SlotDescriptor slot: desc_.getSlots()) {
-        if (!slot.isMaterialized() || slot == countStarSlot_) continue;
-        if (slot.getColumn() == null ||
-            slot.getColumn().getPosition() >= table.getNumClusteringCols()) {
-          ++perHostScanRanges;
-        }
-      }
+      perHostScanRanges = numColumnsReadFromFile;
     } else {
       perHostScanRanges = (int) Math.ceil((
           (double) scanRanges_.size() / (double) numNodes_) * SCAN_RANGE_SKEW_FACTOR);
@@ -1224,12 +1290,13 @@ public class HdfsScanNode extends ScanNode {
     long avgScanRangeBytes = (long) Math.ceil(totalBytes_ / (double) scanRanges_.size());
     // The +1 accounts for an extra I/O buffer to read past the scan range due to a
     // trailing record spanning Hdfs blocks.
-    long readSize = BackendConfig.INSTANCE.getReadSize();
+    long maxIoBufferSize =
+        BitUtil.roundUpToPowerOf2(BackendConfig.INSTANCE.getReadSize());
     long perThreadIoBuffers =
-        Math.min((long) Math.ceil(avgScanRangeBytes / (double) readSize),
+        Math.min((long) Math.ceil(avgScanRangeBytes / (double) maxIoBufferSize),
             MAX_IO_BUFFERS_PER_THREAD) + 1;
     long perInstanceMemEstimate = checkedMultiply(
-        checkedMultiply(maxScannerThreads, perThreadIoBuffers), readSize);
+        checkedMultiply(maxScannerThreads, perThreadIoBuffers), maxIoBufferSize);
 
     // Sanity check: the tighter estimation should not exceed the per-host maximum.
     long perHostUpperBound = getPerHostMemUpperBound();
@@ -1240,7 +1307,51 @@ public class HdfsScanNode extends ScanNode {
       perInstanceMemEstimate = perHostUpperBound;
     }
     perInstanceMemEstimate = Math.max(perInstanceMemEstimate, MIN_MEMORY_ESTIMATE);
-    nodeResourceProfile_ = ResourceProfile.noReservation(perInstanceMemEstimate);
+
+    Pair<Long, Long> reservation = computeReservation(numColumnsReadFromFile);
+    nodeResourceProfile_ = new ResourceProfileBuilder()
+        .setMemEstimateBytes(perInstanceMemEstimate)
+        .setMinReservationBytes(reservation.first).build();
+    idealScanRangeReservation_ = reservation.second;
+  }
+
+  /*
+   *  Compute the minimum and ideal memory reservation to process a single scan range
+   *  (i.e. hdfs split). Bound the reservation based on:
+   * - One minimum-sized buffer per IoMgr scan range, which is the absolute minimum
+   *   required to scan the data.
+   * - A maximum of either 1 or 3 max-sized I/O buffers per IoMgr scan range for
+   *   the minimum and ideal reservation respectively. 1 max-sized I/O buffer avoids
+   *   issuing small I/O unnecessarily while 3 max-sized I/O buffers guarantees higher
+   *   throughput by overlapping compute and I/O efficiently.
+   * - A maximum reservation of the hdfs split size, to avoid reserving excessive
+   *   memory for small files or ranges, e.g. small dimension tables with very few
+   *   rows.
+   */
+  private Pair<Long, Long> computeReservation(int numColumnsReadFromFile) {
+    Preconditions.checkState(maxScanRangeBytes_ >= 0);
+    long maxIoBufferSize =
+        BitUtil.roundUpToPowerOf2(BackendConfig.INSTANCE.getReadSize());
+    // Scanners for columnar formats issue one IoMgr scan range for metadata, followed by
+    // one IoMgr scan range per column in parallel. Scanners for row-oriented formats
+    // issue only one IoMgr scan range at a time.
+    int iomgrScanRangesPerSplit = fileFormats_.contains(HdfsFileFormat.PARQUET) ?
+        Math.max(1, numColumnsReadFromFile) : 1;
+    // Need one buffer per IoMgr scan range to execute the scan.
+    long minReservationToExecute =
+        iomgrScanRangesPerSplit * BackendConfig.INSTANCE.getMinBufferSize();
+
+    // Quantize the max scan range (i.e. hdfs split) size to an I/O buffer size.
+    long quantizedMaxScanRangeBytes = maxScanRangeBytes_ < maxIoBufferSize ?
+        BitUtil.roundUpToPowerOf2(maxScanRangeBytes_) :
+        BitUtil.roundUpToPowerOf2Factor(maxScanRangeBytes_, maxIoBufferSize);
+    long minReservationBytes = Math.max(minReservationToExecute,
+        Math.min(iomgrScanRangesPerSplit * maxIoBufferSize,
+            quantizedMaxScanRangeBytes));
+    long idealReservationBytes = Math.max(minReservationToExecute,
+        Math.min(iomgrScanRangesPerSplit * maxIoBufferSize * 3,
+            quantizedMaxScanRangeBytes));
+    return Pair.create(minReservationBytes, idealReservationBytes);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/fe/src/main/java/org/apache/impala/util/BitUtil.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/BitUtil.java b/fe/src/main/java/org/apache/impala/util/BitUtil.java
index 839dd6e..6bb670d 100644
--- a/fe/src/main/java/org/apache/impala/util/BitUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/BitUtil.java
@@ -29,4 +29,10 @@ public class BitUtil {
   public static long roundUpToPowerOf2(long val) {
     return 1L << log2Ceiling(val);
   }
+
+  // Round up 'val' to the nearest multiple of a power-of-two 'factor'.
+  // 'val' must be > 0.
+  public static long roundUpToPowerOf2Factor(long val, long factor) {
+    return (val + (factor - 1)) & ~(factor - 1);
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/fe/src/test/java/org/apache/impala/util/BitUtilTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/util/BitUtilTest.java b/fe/src/test/java/org/apache/impala/util/BitUtilTest.java
index a134b6a..a6da80c 100644
--- a/fe/src/test/java/org/apache/impala/util/BitUtilTest.java
+++ b/fe/src/test/java/org/apache/impala/util/BitUtilTest.java
@@ -46,4 +46,10 @@ public class BitUtilTest {
     assertEquals(0x8000000000000000L, BitUtil.roundUpToPowerOf2(0x8000000000000000L - 1));
   }
 
+  @Test
+  public void testPowerOf2Factor() {
+    assertEquals(BitUtil.roundUpToPowerOf2Factor(7, 8), 8);
+    assertEquals(BitUtil.roundUpToPowerOf2Factor(8, 8), 8);
+    assertEquals(BitUtil.roundUpToPowerOf2Factor(9, 8), 16);
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
index f25ad0a..533ac42 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/constant-folding.test
@@ -5,7 +5,7 @@ where 5 + 5 < c_custkey and o_orderkey = (2 + 2)
   and (coalesce(2, 3, 4) * 10) + l_linenumber < (0 * 1)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=264.00MB mem-reservation=24.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -59,7 +59,7 @@ PLAN-ROOT SINK
    parquet dictionary predicates: c_custkey > 10
    parquet dictionary predicates on o: o_orderkey = 4
    parquet dictionary predicates on o_lineitems: 20 + l_linenumber < 0
-   mem-estimate=176.00MB mem-reservation=0B
+   mem-estimate=264.00MB mem-reservation=24.00MB
    tuple-ids=0 row-size=24B cardinality=15000
 ====
 # Test HBase scan node.
@@ -107,7 +107,7 @@ having 1024 * 1024 * count(*) % 2 = 0
   and (sm between 5 and 10)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=1.94MB
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=1.97MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -125,7 +125,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=20B cardinality=7300
 ====
 # Test hash join.
@@ -136,7 +136,7 @@ left outer join functional.alltypes b
 where round(1.11 + 2.22 + 3.33 + 4.44, 1) < cast(b.double_col as decimal(3, 2))
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=257.94MB mem-reservation=1.94MB
+|  Per-Host Resources: mem-estimate=257.94MB mem-reservation=2.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -157,7 +157,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet dictionary predicates: CAST(b.double_col AS DECIMAL(3,2)) > 11.1
-|     mem-estimate=128.00MB mem-reservation=0B
+|     mem-estimate=128.00MB mem-reservation=32.00KB
 |     tuple-ids=1 row-size=20B cardinality=730
 |
 00:SCAN HDFS [functional.alltypes a]
@@ -167,7 +167,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=8B cardinality=7300
 ====
 # Test nested-loop join. Same as above but and with a disjunction in the On clause.
@@ -179,7 +179,7 @@ left outer join functional.alltypes b
 where cast(b.double_col as decimal(3, 2)) > round(1.11 + 2.22 + 3.33 + 4.44, 1)
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=256.01MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=256.01MB mem-reservation=64.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -198,7 +198,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet dictionary predicates: CAST(b.double_col AS DECIMAL(3,2)) > 11.1
-|     mem-estimate=128.00MB mem-reservation=0B
+|     mem-estimate=128.00MB mem-reservation=32.00KB
 |     tuple-ids=1 row-size=20B cardinality=730
 |
 00:SCAN HDFS [functional.alltypes a]
@@ -208,7 +208,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=8B cardinality=7300
 ====
 # Test distinct aggregation with grouping.
@@ -242,7 +242,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=20B cardinality=7300
 ====
 # Test non-grouping distinct aggregation.
@@ -251,7 +251,7 @@ from functional.alltypes
 having 1024 * 1024 * count(*) % 2 = 0
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=1.94MB
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=1.97MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -274,7 +274,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=4B cardinality=7300
 ====
 # Test analytic eval node.
@@ -284,7 +284,7 @@ select first_value(1 + 1 + int_col - (1 - 1)) over
 from functional.alltypes
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=16.00MB
+|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=16.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -309,7 +309,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=29B cardinality=7300
 ====
 # Test sort node.
@@ -317,7 +317,7 @@ select int_col from functional.alltypes
 order by id * abs((factorial(5) / power(2, 4)))
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=134.00MB mem-reservation=6.00MB
+|  Per-Host Resources: mem-estimate=134.00MB mem-reservation=6.03MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -333,7 +333,7 @@ PLAN-ROOT SINK
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=8B cardinality=7300
 ====
 # Test HDFS table sink.
@@ -342,7 +342,7 @@ select id, int_col, cast(1 + 1 + 1 + year as int), cast(month - (1 - 1 - 1) as i
 from functional.alltypessmall
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB
 WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(CAST(3 + year AS INT),CAST(month - -1 AS INT))]
 |  partitions=4
 |  mem-estimate=1.56KB mem-reservation=0B
@@ -354,7 +354,7 @@ WRITE TO HDFS [functional.alltypes, OVERWRITE=false, PARTITION-KEYS=(CAST(3 + ye
      partitions: 4/4 rows=100
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=8.00KB
    tuple-ids=0 row-size=16B cardinality=100
 ====
 # Constant folding does not work across query blocks.
@@ -366,7 +366,7 @@ select sum(id + c3) from
   ) v3
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=32.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -383,6 +383,6 @@ PLAN-ROOT SINK
      columns: all
    extrapolated-rows=disabled
    limit: 2
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=4B cardinality=2
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test b/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test
index 0f4a5da..3d234d1 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/disable-codegen.test
@@ -1,7 +1,7 @@
 # Rows per node is < 3000: codegen should be disabled.
 select count(*) from functional.alltypes
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=32.00KB
 Per-Host Resource Estimates: Memory=148.00MB
 Codegen disabled by planner
 
@@ -21,7 +21,7 @@ PLAN-ROOT SINK
 # Rows per node is > 3000: codegen should be enabled.
 select count(*) from functional.alltypesagg
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=128.00KB
 Per-Host Resource Estimates: Memory=100.00MB
 
 PLAN-ROOT SINK
@@ -40,7 +40,7 @@ PLAN-ROOT SINK
 # No stats on functional_parquet: codegen should be disabled.
 select count(*) from functional_parquet.alltypes
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=8.00KB
 Per-Host Resource Estimates: Memory=21.00MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
@@ -56,12 +56,12 @@ PLAN-ROOT SINK
 |  output: sum_init_zero(functional_parquet.alltypes.parquet-stats: num_rows)
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=174.39KB
+   partitions=24/24 files=24 size=174.62KB
 ====
 # > 3000 rows returned to coordinator: codegen should be enabled
 select * from functional_parquet.alltypes
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=88.00KB
 Per-Host Resource Estimates: Memory=128.00MB
 WARNING: The following tables are missing relevant table and/or column statistics.
 functional_parquet.alltypes
@@ -71,14 +71,14 @@ PLAN-ROOT SINK
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=174.39KB
+   partitions=24/24 files=24 size=174.62KB
 ====
 # Optimisation is enabled for join producing < 3000 rows
 select count(*)
 from functional.alltypes t1
 join functional.alltypestiny t2 on t1.id = t2.id
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=2.94MB
+Max Per-Host Resource Reservation: Memory=2.98MB
 Per-Host Resource Estimates: Memory=182.94MB
 Codegen disabled by planner
 
@@ -108,7 +108,7 @@ PLAN-ROOT SINK
 # Optimisation is disabled by cross join producing > 3000 rows
 select count(*) from functional.alltypes t1, functional.alltypes t2
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=64.00KB
 Per-Host Resource Estimates: Memory=276.00MB
 
 PLAN-ROOT SINK
@@ -137,7 +137,7 @@ select count(*) from (
   union all
   select * from functional.alltypestiny) v
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=32.00KB
 Per-Host Resource Estimates: Memory=148.00MB
 Codegen disabled by planner
 
@@ -166,7 +166,7 @@ select count(*) from (
   union all
   select * from functional.alltypes) v
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=32.00KB
 Per-Host Resource Estimates: Memory=148.00MB
 
 PLAN-ROOT SINK
@@ -193,7 +193,7 @@ PLAN-ROOT SINK
 select sum(l_discount)
 from (select * from tpch.lineitem limit 1000) v
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=8.00MB
 Per-Host Resource Estimates: Memory=274.00MB
 Codegen disabled by planner
 
@@ -214,7 +214,7 @@ PLAN-ROOT SINK
 select sum(l_discount)
 from (select * from tpch.lineitem where l_orderkey > 100 limit 1000) v
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=0B
+Max Per-Host Resource Reservation: Memory=8.00MB
 Per-Host Resource Estimates: Memory=274.00MB
 
 PLAN-ROOT SINK

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
index bb12bca..55439d6 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/fk-pk-join-detection.test
@@ -5,7 +5,7 @@ on ss_customer_sk = c_customer_sk
 where c_salutation = 'Mrs.'
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=185.50MB mem-reservation=9.50MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=185.50MB mem-reservation=25.50MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -24,7 +24,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet dictionary predicates: c_salutation = 'Mrs.'
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=255B cardinality=16667
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -35,7 +35,7 @@ PLAN-ROOT SINK
      partitions: 1824/1824 rows=2880404
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=100B cardinality=2880404
 ====
 # Single-column FK/PK join detection on left outer join. The join cardinality
@@ -46,7 +46,7 @@ on ss_customer_sk = c_customer_sk
 where c_salutation = 'Mrs.'
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=184.50MB mem-reservation=8.50MB
+|  Per-Host Resources: mem-estimate=184.50MB mem-reservation=24.50MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -65,7 +65,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet dictionary predicates: c_salutation = 'Mrs.'
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=255B cardinality=16667
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -75,7 +75,7 @@ PLAN-ROOT SINK
      partitions: 1824/1824 rows=2880404
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=100B cardinality=2880404
 ====
 # Single-column FK/PK join detection on right outer join. The join cardinality
@@ -86,7 +86,7 @@ on ss_customer_sk = c_customer_sk
 where c_salutation = 'Mrs.'
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=185.50MB mem-reservation=9.50MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=185.50MB mem-reservation=25.50MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -105,7 +105,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet dictionary predicates: c_salutation = 'Mrs.'
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=255B cardinality=16667
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -116,7 +116,7 @@ PLAN-ROOT SINK
      partitions: 1824/1824 rows=2880404
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=100B cardinality=2880404
 ====
 # Multi-column FK/PK join detection
@@ -126,7 +126,7 @@ on ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number
 where sr_return_quantity < 10
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=214.75MB mem-reservation=6.75MB runtime-filters-memory=2.00MB
+|  Per-Host Resources: mem-estimate=214.75MB mem-reservation=22.75MB runtime-filters-memory=2.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -145,7 +145,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet dictionary predicates: sr_return_quantity < 10
-|     mem-estimate=80.00MB mem-reservation=0B
+|     mem-estimate=80.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=88B cardinality=28751
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -156,7 +156,7 @@ PLAN-ROOT SINK
      partitions: 1824/1824 rows=2880404
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=100B cardinality=2880404
 ====
 # Many-to-many join detection.
@@ -165,7 +165,7 @@ tpcds.store_sales inner join tpcds.web_sales
 on ss_sold_time_sk = ws_sold_time_sk
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=397.67MB mem-reservation=35.00MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=397.67MB mem-reservation=51.00MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -182,7 +182,7 @@ PLAN-ROOT SINK
 |       table: rows=719384 size=140.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=160.00MB mem-reservation=0B
+|     mem-estimate=160.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=144B cardinality=719384
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -193,7 +193,7 @@ PLAN-ROOT SINK
      partitions: 1824/1824 rows=2880404
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=100B cardinality=2880404
 ====
 # PK/PK join is detected as FK/PK.
@@ -203,7 +203,7 @@ on a.d_date_sk = b.d_date_sk
 where a.d_holiday = "Y"
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=114.00MB mem-reservation=18.00MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=114.00MB mem-reservation=34.00MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -222,7 +222,7 @@ PLAN-ROOT SINK
 |       columns: all
 |     extrapolated-rows=disabled
 |     parquet dictionary predicates: a.d_holiday = 'Y'
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=0 row-size=303B cardinality=36525
 |
 01:SCAN HDFS [tpcds.date_dim b]
@@ -232,7 +232,7 @@ PLAN-ROOT SINK
      table: rows=73049 size=9.84MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=48.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=8.00MB
    tuple-ids=1 row-size=303B cardinality=73049
 ====
 # Single query with various join types combined.
@@ -246,7 +246,7 @@ where ss_item_sk = sr_item_sk and ss_ticket_number = sr_ticket_number
   and d1.d_fy_week_seq = 1000
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=362.81MB mem-reservation=12.75MB runtime-filters-memory=5.00MB
+|  Per-Host Resources: mem-estimate=362.81MB mem-reservation=50.81MB runtime-filters-memory=5.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -263,7 +263,7 @@ PLAN-ROOT SINK
 |       table: rows=100000 size=12.60MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=2 row-size=4B cardinality=100000
 |
 07:HASH JOIN [INNER JOIN]
@@ -279,7 +279,7 @@ PLAN-ROOT SINK
 |       table: rows=73049 size=9.84MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=4 row-size=4B cardinality=73049
 |
 06:HASH JOIN [INNER JOIN]
@@ -304,7 +304,7 @@ PLAN-ROOT SINK
 |  |       columns: all
 |  |     extrapolated-rows=disabled
 |  |     parquet dictionary predicates: d1.d_fy_week_seq = 1000
-|  |     mem-estimate=48.00MB mem-reservation=0B
+|  |     mem-estimate=48.00MB mem-reservation=8.00MB
 |  |     tuple-ids=3 row-size=8B cardinality=7
 |  |
 |  00:SCAN HDFS [tpcds.store_sales]
@@ -315,7 +315,7 @@ PLAN-ROOT SINK
 |       partitions: 1824/1824 rows=2880404
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=128.00MB mem-reservation=0B
+|     mem-estimate=128.00MB mem-reservation=8.00MB
 |     tuple-ids=0 row-size=24B cardinality=2880404
 |
 01:SCAN HDFS [tpcds.store_returns]
@@ -325,7 +325,7 @@ PLAN-ROOT SINK
      table: rows=287514 size=31.19MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=8.00MB
    tuple-ids=1 row-size=20B cardinality=287514
 ====
 # Assumed FK/PK join becasue of non-trivial equi-join exprs.
@@ -334,7 +334,7 @@ tpcds.store_sales inner join tpcds.customer
 on ss_customer_sk % 10 = c_customer_sk / 100
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=211.00MB mem-reservation=35.00MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=211.00MB mem-reservation=51.00MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -351,7 +351,7 @@ PLAN-ROOT SINK
 |       table: rows=100000 size=12.60MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=255B cardinality=100000
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -362,7 +362,7 @@ PLAN-ROOT SINK
      partitions: 1824/1824 rows=2880404
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=100B cardinality=2880404
 ====
 # Assumed FK/PK join due to missing stats on the rhs. Join cardinality is equal to
@@ -372,7 +372,7 @@ tpcds.store_sales inner join tpcds_seq_snap.customer
 on ss_customer_sk = c_customer_sk
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=2.17GB mem-reservation=35.00MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=2.17GB mem-reservation=51.00MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -384,12 +384,12 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=8B cardinality=2880404
 |
 |--01:SCAN HDFS [tpcds_seq_snap.customer]
-|     partitions=1/1 files=1 size=8.59MB
+|     partitions=1/1 files=1 size=8.58MB
 |     stored statistics:
-|       table: rows=unavailable size=8.59MB
+|       table: rows=unavailable size=8.58MB
 |       columns: unavailable
 |     extrapolated-rows=disabled
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=4B cardinality=unavailable
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -400,7 +400,7 @@ PLAN-ROOT SINK
      partitions: 1824/1824 rows=2880404
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=4B cardinality=2880404
 ====
 # Assumed FK/PK join due to missing stats on the lhs. Join cardinality is unknown.
@@ -409,7 +409,7 @@ tpcds_seq_snap.store_sales inner join tpcds.customer
 on ss_customer_sk = c_customer_sk
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=178.94MB mem-reservation=2.94MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=178.94MB mem-reservation=18.94MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -426,18 +426,18 @@ PLAN-ROOT SINK
 |       table: rows=100000 size=12.60MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=48.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=4B cardinality=100000
 |
 00:SCAN HDFS [tpcds_seq_snap.store_sales]
-   partitions=1824/1824 files=1824 size=207.90MB
+   partitions=1824/1824 files=1824 size=207.85MB
    runtime filters: RF000[bloom] -> ss_customer_sk
    stored statistics:
      table: rows=unavailable size=unavailable
      partitions: 0/1824 rows=unavailable
      columns: unavailable
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=4B cardinality=unavailable
 ====
 # Join is detected as many-to-many even though the rhs join columns
@@ -448,7 +448,7 @@ tpcds.store_sales inner join
 on ss_sold_time_sk = ws_sold_time_sk
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=299.00MB mem-reservation=4.88MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=299.00MB mem-reservation=18.94MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -470,7 +470,7 @@ PLAN-ROOT SINK
 |       table: rows=719384 size=140.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=160.00MB mem-reservation=0B
+|     mem-estimate=160.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=4B cardinality=719384
 |
 00:SCAN HDFS [tpcds.store_sales]
@@ -481,6 +481,6 @@ PLAN-ROOT SINK
      partitions: 1824/1824 rows=2880404
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=100B cardinality=2880404
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
index 96015e0..1df9270 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/max-row-size.test
@@ -4,7 +4,7 @@ select straight_join *
 from tpch_parquet.customer
     inner join tpch_parquet.nation on c_nationkey = n_nationkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=17.94MB
+Max Per-Host Resource Reservation: Memory=33.97MB
 Per-Host Resource Estimates: Memory=57.94MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -17,7 +17,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=355B cardinality=150000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-Per-Host Resources: mem-estimate=41.94MB mem-reservation=17.94MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=41.94MB mem-reservation=33.94MB runtime-filters-memory=1.00MB
 02:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: c_nationkey = n_nationkey
 |  fk/pk conjuncts: c_nationkey = n_nationkey
@@ -30,14 +30,14 @@ Per-Host Resources: mem-estimate=41.94MB mem-reservation=17.94MB runtime-filters
 |  |  tuple-ids=1 row-size=117B cardinality=25
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB
 |  01:SCAN HDFS [tpch_parquet.nation, RANDOM]
 |     partitions=1/1 files=1 size=2.74KB
 |     stored statistics:
 |       table: rows=25 size=2.74KB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=16.00MB mem-reservation=0B
+|     mem-estimate=16.00MB mem-reservation=32.00KB
 |     tuple-ids=1 row-size=117B cardinality=25
 |
 00:SCAN HDFS [tpch_parquet.customer, RANDOM]
@@ -47,7 +47,7 @@ Per-Host Resources: mem-estimate=41.94MB mem-reservation=17.94MB runtime-filters
      table: rows=150000 size=12.31MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=24.00MB mem-reservation=0B
+   mem-estimate=24.00MB mem-reservation=16.00MB
    tuple-ids=0 row-size=238B cardinality=150000
 ====
 # Join with large build side.
@@ -56,8 +56,8 @@ select straight_join *
 from tpch_parquet.lineitem
     left join tpch_parquet.orders on l_orderkey = o_orderkey
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=46.00MB
-Per-Host Resource Estimates: Memory=420.41MB
+Max Per-Host Resource Reservation: Memory=166.00MB
+Per-Host Resource Estimates: Memory=428.41MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
 |  Per-Host Resources: mem-estimate=0B mem-reservation=0B
@@ -69,7 +69,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1N row-size=454B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=380.41MB mem-reservation=46.00MB
+Per-Host Resources: mem-estimate=380.41MB mem-reservation=118.00MB
 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
 |  hash predicates: l_orderkey = o_orderkey
 |  fk/pk conjuncts: l_orderkey = o_orderkey
@@ -81,23 +81,23 @@ Per-Host Resources: mem-estimate=380.41MB mem-reservation=46.00MB
 |  |  tuple-ids=1 row-size=191B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=48.00MB mem-reservation=48.00MB
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     partitions=1/1 files=2 size=54.07MB
 |     stored statistics:
 |       table: rows=1500000 size=54.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=40.00MB mem-reservation=0B
+|     mem-estimate=48.00MB mem-reservation=48.00MB
 |     tuple-ids=1 row-size=191B cardinality=1500000
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Null-aware anti-join with medium build side.
@@ -105,7 +105,7 @@ Per-Host Resources: mem-estimate=380.41MB mem-reservation=46.00MB
 select * from tpch_parquet.lineitem
 where l_orderkey not in (select o_orderkey from tpch_parquet.orders)
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=34.00MB
+Max Per-Host Resource Reservation: Memory=114.00MB
 Per-Host Resource Estimates: Memory=154.00MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -118,7 +118,7 @@ PLAN-ROOT SINK
 |  tuple-ids=0 row-size=263B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=114.00MB mem-reservation=34.00MB
+Per-Host Resources: mem-estimate=114.00MB mem-reservation=106.00MB
 02:HASH JOIN [NULL AWARE LEFT ANTI JOIN, BROADCAST]
 |  hash predicates: l_orderkey = o_orderkey
 |  mem-estimate=34.00MB mem-reservation=34.00MB spill-buffer=1.00MB
@@ -129,23 +129,23 @@ Per-Host Resources: mem-estimate=114.00MB mem-reservation=34.00MB
 |  |  tuple-ids=1 row-size=8B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=8.00MB
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     partitions=1/1 files=2 size=54.07MB
 |     stored statistics:
 |       table: rows=1500000 size=54.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=40.00MB mem-reservation=0B
+|     mem-estimate=40.00MB mem-reservation=8.00MB
 |     tuple-ids=1 row-size=8B cardinality=1500000
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # Mid NDV aggregation.
@@ -156,7 +156,7 @@ from tpch_parquet.lineitem
 group by 1, 2
 having count(*) = 1
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=113.00MB
+Max Per-Host Resource Reservation: Memory=137.00MB
 Per-Host Resource Estimates: Memory=253.12MB
 
 F04:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -201,14 +201,14 @@ Per-Host Resources: mem-estimate=86.12MB mem-reservation=66.00MB runtime-filters
 |  |  tuple-ids=1 row-size=25B cardinality=1500000
 |  |
 |  F01:PLAN FRAGMENT [RANDOM] hosts=2 instances=2
-|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=40.00MB mem-reservation=16.00MB
 |  01:SCAN HDFS [tpch_parquet.orders, RANDOM]
 |     partitions=1/1 files=2 size=54.07MB
 |     stored statistics:
 |       table: rows=1500000 size=54.07MB
 |       columns: all
 |     extrapolated-rows=disabled
-|     mem-estimate=40.00MB mem-reservation=0B
+|     mem-estimate=40.00MB mem-reservation=16.00MB
 |     tuple-ids=1 row-size=25B cardinality=1500000
 |
 04:EXCHANGE [HASH(l_orderkey)]
@@ -216,15 +216,15 @@ Per-Host Resources: mem-estimate=86.12MB mem-reservation=66.00MB runtime-filters
 |  tuple-ids=0 row-size=8B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=81.00MB mem-reservation=1.00MB runtime-filters-memory=1.00MB
+Per-Host Resources: mem-estimate=81.00MB mem-reservation=9.00MB runtime-filters-memory=1.00MB
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    runtime filters: RF000[bloom] -> l_orderkey
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=8.00MB
    tuple-ids=0 row-size=8B cardinality=6001215
 ====
 # High NDV aggregation.
@@ -232,7 +232,7 @@ Per-Host Resources: mem-estimate=81.00MB mem-reservation=1.00MB runtime-filters-
 select distinct *
 from tpch_parquet.lineitem
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=80.00MB
+Max Per-Host Resource Reservation: Memory=152.00MB
 Per-Host Resource Estimates: Memory=3.31GB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -256,19 +256,19 @@ Per-Host Resources: mem-estimate=1.62GB mem-reservation=46.00MB
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=1.69GB mem-reservation=34.00MB
+Per-Host Resources: mem-estimate=1.69GB mem-reservation=106.00MB
 01:AGGREGATE [STREAMING]
 |  group by: tpch_parquet.lineitem.l_orderkey, tpch_parquet.lineitem.l_partkey, tpch_parquet.lineitem.l_suppkey, tpch_parquet.lineitem.l_linenumber, tpch_parquet.lineitem.l_quantity, tpch_parquet.lineitem.l_extendedprice, tpch_parquet.lineitem.l_discount, tpch_parquet.lineitem.l_tax, tpch_parquet.lineitem.l_returnflag, tpch_parquet.lineitem.l_linestatus, tpch_parquet.lineitem.l_shipdate, tpch_parquet.lineitem.l_commitdate, tpch_parquet.lineitem.l_receiptdate, tpch_parquet.lineitem.l_shipinstruct, tpch_parquet.lineitem.l_shipmode, tpch_parquet.lineitem.l_comment
 |  mem-estimate=1.62GB mem-reservation=34.00MB spill-buffer=2.00MB
 |  tuple-ids=1 row-size=263B cardinality=6001215
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=72.00MB
    tuple-ids=0 row-size=263B cardinality=6001215
 ====
 # High NDV aggregation with string aggregation function.
@@ -277,7 +277,7 @@ select l_orderkey, l_partkey, group_concat(l_linestatus, ",")
 from tpch_parquet.lineitem
 group by 1, 2
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=82.00MB
+Max Per-Host Resource Reservation: Memory=106.00MB
 Per-Host Resource Estimates: Memory=482.91MB
 
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -302,7 +302,7 @@ Per-Host Resources: mem-estimate=201.46MB mem-reservation=48.00MB
 |  tuple-ids=1 row-size=32B cardinality=6001215
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=281.46MB mem-reservation=34.00MB
+Per-Host Resources: mem-estimate=281.46MB mem-reservation=58.00MB
 01:AGGREGATE [STREAMING]
 |  output: group_concat(l_linestatus, ',')
 |  group by: l_orderkey, l_partkey
@@ -310,12 +310,12 @@ Per-Host Resources: mem-estimate=281.46MB mem-reservation=34.00MB
 |  tuple-ids=1 row-size=32B cardinality=6001215
 |
 00:SCAN HDFS [tpch_parquet.lineitem, RANDOM]
-   partitions=1/1 files=3 size=193.73MB
+   partitions=1/1 files=3 size=193.72MB
    stored statistics:
-     table: rows=6001215 size=193.73MB
+     table: rows=6001215 size=193.72MB
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=80.00MB mem-reservation=0B
+   mem-estimate=80.00MB mem-reservation=24.00MB
    tuple-ids=0 row-size=33B cardinality=6001215
 ====
 # Sort + Analytic.
@@ -323,7 +323,7 @@ Per-Host Resources: mem-estimate=281.46MB mem-reservation=34.00MB
 select max(tinyint_col) over(partition by int_col)
 from functional.alltypes
 ---- DISTRIBUTEDPLAN
-Max Per-Host Resource Reservation: Memory=40.00MB
+Max Per-Host Resource Reservation: Memory=40.03MB
 Per-Host Resource Estimates: Memory=56.00MB
 Codegen disabled by planner
 
@@ -354,7 +354,7 @@ Per-Host Resources: mem-estimate=40.00MB mem-reservation=40.00MB
 |  tuple-ids=0 row-size=5B cardinality=7300
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=3
-Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=16.00MB mem-reservation=32.00KB
 00:SCAN HDFS [functional.alltypes, RANDOM]
    partitions=24/24 files=24 size=478.45KB
    stored statistics:
@@ -362,6 +362,6 @@ Per-Host Resources: mem-estimate=16.00MB mem-reservation=0B
      partitions: 24/24 rows=7300
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=32.00KB
    tuple-ids=0 row-size=5B cardinality=7300
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
index 7f6d96b..8dd2593 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/min-max-runtime-filters.test
@@ -99,7 +99,7 @@ select count(*) from functional_kudu.alltypes a, functional_parquet.alltypes b,
 where a.int_col = b.int_col and a.int_col = c.int_col
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=2.02GB mem-reservation=36.94MB runtime-filters-memory=1.00MB
+|  Per-Host Resources: mem-estimate=2.02GB mem-reservation=36.95MB runtime-filters-memory=1.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -127,14 +127,14 @@ PLAN-ROOT SINK
 |  tuple-ids=0,1 row-size=8B cardinality=7300
 |
 |--01:SCAN HDFS [functional_parquet.alltypes b]
-|     partitions=24/24 files=24 size=174.39KB
+|     partitions=24/24 files=24 size=174.62KB
 |     runtime filters: RF000[bloom] -> b.int_col
 |     stored statistics:
 |       table: rows=unavailable size=unavailable
 |       partitions: 0/24 rows=unavailable
 |       columns: unavailable
 |     extrapolated-rows=disabled
-|     mem-estimate=16.00MB mem-reservation=0B
+|     mem-estimate=16.00MB mem-reservation=8.00KB
 |     tuple-ids=1 row-size=4B cardinality=unavailable
 |
 00:SCAN KUDU [functional_kudu.alltypes a]

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
index 61d646b..31129ff 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mt-dop-validation.test
@@ -40,7 +40,7 @@ order by cnt, bigint_col
 limit 10
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.00MB
+|  Per-Host Resources: mem-estimate=144.00MB mem-reservation=34.02MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -56,7 +56,7 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=16B cardinality=unavailable
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=179.19KB
+   partitions=24/24 files=24 size=174.39KB
    predicates: id < 10
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -65,7 +65,7 @@ PLAN-ROOT SINK
    extrapolated-rows=disabled
    parquet statistics predicates: id < 10
    parquet dictionary predicates: id < 10
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=24.00KB
    tuple-ids=0 row-size=16B cardinality=unavailable
 ---- PARALLELPLANS
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -97,7 +97,7 @@ Per-Host Resources: mem-estimate=384.00MB mem-reservation=102.00MB
 |  tuple-ids=1 row-size=16B cardinality=unavailable
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
-Per-Host Resources: mem-estimate=432.00MB mem-reservation=102.00MB
+Per-Host Resources: mem-estimate=432.00MB mem-reservation=102.07MB
 01:AGGREGATE [STREAMING]
 |  output: count(int_col)
 |  group by: bigint_col
@@ -105,7 +105,7 @@ Per-Host Resources: mem-estimate=432.00MB mem-reservation=102.00MB
 |  tuple-ids=1 row-size=16B cardinality=unavailable
 |
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   partitions=24/24 files=24 size=179.19KB
+   partitions=24/24 files=24 size=174.39KB
    predicates: id < 10
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -114,7 +114,7 @@ Per-Host Resources: mem-estimate=432.00MB mem-reservation=102.00MB
    extrapolated-rows=disabled
    parquet statistics predicates: id < 10
    parquet dictionary predicates: id < 10
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=24.00KB
    tuple-ids=0 row-size=16B cardinality=unavailable
 ====
 # Single-table scan/filter/analytic should work.
@@ -123,7 +123,7 @@ from functional_parquet.alltypes
 where id < 10
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=10.00MB
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=10.02MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -141,7 +141,7 @@ PLAN-ROOT SINK
 |  tuple-ids=4 row-size=8B cardinality=unavailable
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=179.19KB
+   partitions=24/24 files=24 size=174.39KB
    predicates: id < 10
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -150,7 +150,7 @@ PLAN-ROOT SINK
    extrapolated-rows=disabled
    parquet statistics predicates: id < 10
    parquet dictionary predicates: id < 10
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=16.00KB
    tuple-ids=0 row-size=8B cardinality=unavailable
 ---- PARALLELPLANS
 F02:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -182,9 +182,9 @@ Per-Host Resources: mem-estimate=30.00MB mem-reservation=30.00MB
 |  tuple-ids=0 row-size=8B cardinality=unavailable
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
-Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=48.00MB mem-reservation=48.00KB
 00:SCAN HDFS [functional_parquet.alltypes, RANDOM]
-   partitions=24/24 files=24 size=179.19KB
+   partitions=24/24 files=24 size=174.39KB
    predicates: id < 10
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -193,7 +193,7 @@ Per-Host Resources: mem-estimate=48.00MB mem-reservation=0B
    extrapolated-rows=disabled
    parquet statistics predicates: id < 10
    parquet dictionary predicates: id < 10
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=16.00KB
    tuple-ids=0 row-size=8B cardinality=unavailable
 ====
 # Nested-loop join in a subplan should work.
@@ -202,7 +202,7 @@ from tpch_nested_parquet.customer c, c.c_orders o, o.o_lineitems
 where c_custkey < 10 and o_orderkey < 5 and l_linenumber < 3
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=88.00MB mem-reservation=88.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -257,7 +257,7 @@ PLAN-ROOT SINK
    parquet dictionary predicates: c_custkey < 10
    parquet dictionary predicates on o: o_orderkey < 5
    parquet dictionary predicates on o_lineitems: l_linenumber < 3
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=88.00MB
    tuple-ids=0 row-size=254B cardinality=15000
 ---- PARALLELPLANS
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -270,7 +270,7 @@ PLAN-ROOT SINK
 |  tuple-ids=2,1,0 row-size=562B cardinality=1500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
-Per-Host Resources: mem-estimate=264.00MB mem-reservation=0B
+Per-Host Resources: mem-estimate=264.00MB mem-reservation=264.00MB
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=2,1,0 row-size=562B cardinality=1500000
@@ -322,7 +322,7 @@ Per-Host Resources: mem-estimate=264.00MB mem-reservation=0B
    parquet dictionary predicates: c_custkey < 10
    parquet dictionary predicates on o: o_orderkey < 5
    parquet dictionary predicates on o_lineitems: l_linenumber < 3
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=88.00MB
    tuple-ids=0 row-size=254B cardinality=15000
 ====
 # Hash-join in a subplan should work.
@@ -331,7 +331,7 @@ from tpch_nested_parquet.customer c, c.c_orders o1, c.c_orders o2
 where o1.o_orderkey = o2.o_orderkey + 2 and o1.o_orderkey < 5
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=89.94MB mem-reservation=1.94MB
+|  Per-Host Resources: mem-estimate=89.94MB mem-reservation=81.94MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -374,7 +374,7 @@ PLAN-ROOT SINK
    extrapolated-rows=disabled
    parquet statistics predicates on o1: o1.o_orderkey < 5
    parquet dictionary predicates on o1: o1.o_orderkey < 5
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=80.00MB
    tuple-ids=0 row-size=270B cardinality=150000
 ---- PARALLELPLANS
 F01:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
@@ -387,7 +387,7 @@ PLAN-ROOT SINK
 |  tuple-ids=1,0,2 row-size=286B cardinality=1500000
 |
 F00:PLAN FRAGMENT [RANDOM] hosts=3 instances=9
-Per-Host Resources: mem-estimate=269.81MB mem-reservation=5.81MB
+Per-Host Resources: mem-estimate=269.81MB mem-reservation=245.81MB
 01:SUBPLAN
 |  mem-estimate=0B mem-reservation=0B
 |  tuple-ids=1,0,2 row-size=286B cardinality=1500000
@@ -427,6 +427,6 @@ Per-Host Resources: mem-estimate=269.81MB mem-reservation=5.81MB
    extrapolated-rows=disabled
    parquet statistics predicates on o1: o1.o_orderkey < 5
    parquet dictionary predicates on o1: o1.o_orderkey < 5
-   mem-estimate=88.00MB mem-reservation=0B
+   mem-estimate=88.00MB mem-reservation=80.00MB
    tuple-ids=0 row-size=270B cardinality=150000
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
index 2b602c9..3e812c6 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/parquet-filtering.test
@@ -8,7 +8,7 @@ where int_col > 1 and int_col * rand() > 50 and int_col is null
 and int_col > tinyint_col;
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=42.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=42.00MB mem-reservation=16.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -18,7 +18,7 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=179.19KB
+   partitions=24/24 files=24 size=174.39KB
    predicates: int_col IS NULL, int_col > 1, int_col > tinyint_col, int_col * rand() > 50
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -27,7 +27,7 @@ PLAN-ROOT SINK
    extrapolated-rows=disabled
    parquet statistics predicates: int_col > 1
    parquet dictionary predicates: int_col > 1
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=16.00KB
    tuple-ids=0 row-size=5B cardinality=unavailable
 ====
 # Test a variety of types
@@ -40,7 +40,7 @@ and timestamp_cmp(timestamp_col, '2016-11-20 00:00:00') = 1
 and year > 2000 and month < 12;
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=138.00MB mem-reservation=88.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -50,7 +50,7 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=22/24 files=22 size=164.09KB
+   partitions=22/24 files=22 size=159.69KB
    predicates: bool_col, bigint_col < 5000, double_col > 100.00, float_col > 50.00, id = 1, tinyint_col < 50, string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (1, 2, 3, 4, 5), mod(int_col, 2) = 1, timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = 1, date_string_col > '1993-10-01'
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -59,7 +59,7 @@ PLAN-ROOT SINK
    extrapolated-rows=disabled
    parquet statistics predicates: bigint_col < 5000, double_col > 100.00, float_col > 50.00, id = 1, tinyint_col < 50, string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (1, 2, 3, 4, 5), date_string_col > '1993-10-01'
    parquet dictionary predicates: bool_col, bigint_col < 5000, double_col > 100.00, float_col > 50.00, id = 1, tinyint_col < 50, string_col IN ('aaaa', 'bbbb', 'cccc'), smallint_col IN (1, 2, 3, 4, 5), mod(int_col, 2) = 1, timestamp_cmp(timestamp_col, TIMESTAMP '2016-11-20 00:00:00') = 1, date_string_col > '1993-10-01'
-   mem-estimate=128.00MB mem-reservation=0B
+   mem-estimate=128.00MB mem-reservation=88.00KB
    tuple-ids=0 row-size=80B cardinality=unavailable
 ====
 # Test negative cases for IN predicate min/max filtering
@@ -73,7 +73,7 @@ and mod(int_col,50) IN (0,1)
 and id IN (int_col);
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=58.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=58.00MB mem-reservation=24.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -83,7 +83,7 @@ PLAN-ROOT SINK
 |  tuple-ids=1 row-size=8B cardinality=1
 |
 00:SCAN HDFS [functional_parquet.alltypes]
-   partitions=24/24 files=24 size=179.19KB
+   partitions=24/24 files=24 size=174.39KB
    predicates: id IN (int_col), id NOT IN (0, 1, 2), string_col IN ('aaaa', 'bbbb', 'cccc', NULL), mod(int_col, 50) IN (0, 1)
    stored statistics:
      table: rows=unavailable size=unavailable
@@ -91,7 +91,7 @@ PLAN-ROOT SINK
      columns: unavailable
    extrapolated-rows=disabled
    parquet dictionary predicates: id NOT IN (0, 1, 2), string_col IN ('aaaa', 'bbbb', 'cccc', NULL), mod(int_col, 50) IN (0, 1)
-   mem-estimate=48.00MB mem-reservation=0B
+   mem-estimate=48.00MB mem-reservation=24.00KB
    tuple-ids=0 row-size=24B cardinality=unavailable
 ====
 # Test collection types where all collections on the path are required (inner
@@ -101,7 +101,7 @@ select id from functional_parquet.complextypestbl c, c.nested_struct.c.d cn, cn.
 where a.item.e < -10;
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -152,7 +152,7 @@ PLAN-ROOT SINK
    extrapolated-rows=disabled
    parquet statistics predicates on a: a.item.e < -10
    parquet dictionary predicates on a: a.item.e < -10
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=16.00KB
    tuple-ids=0 row-size=24B cardinality=unavailable
 ====
 # Test collection types where the lower collection in the path is optional
@@ -164,7 +164,7 @@ left outer join cn.item a
 where a.item.e < -10;
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -213,7 +213,7 @@ PLAN-ROOT SINK
      table: rows=unavailable size=unavailable
      columns missing stats: id
    extrapolated-rows=disabled
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=16.00KB
    tuple-ids=0 row-size=24B cardinality=unavailable
 ====
 # Tests collection types where the outer is optional (outer join descent)
@@ -223,7 +223,7 @@ select id from functional_parquet.complextypestbl c
 left outer join c.nested_struct.c.d cn, cn.item a where a.item.e < -10;
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=16.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -270,7 +270,7 @@ PLAN-ROOT SINK
      table: rows=unavailable size=unavailable
      columns missing stats: id
    extrapolated-rows=disabled
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=16.00KB
    tuple-ids=0 row-size=24B cardinality=unavailable
 ====
 # Test collections so that each level has a filter applied.
@@ -278,7 +278,7 @@ select c_custkey from tpch_nested_parquet.customer c, c.c_orders o,
 o.o_lineitems l where c_custkey > 0 and o.o_orderkey > 0 and l.l_partkey > 0;
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=264.00MB mem-reservation=24.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -333,7 +333,7 @@ PLAN-ROOT SINK
    parquet dictionary predicates: c_custkey > 0
    parquet dictionary predicates on o: o.o_orderkey > 0
    parquet dictionary predicates on l: l.l_partkey > 0
-   mem-estimate=176.00MB mem-reservation=0B
+   mem-estimate=264.00MB mem-reservation=24.00MB
    tuple-ids=0 row-size=24B cardinality=15000
 ====
 # Test collections in a way that would incorrectly apply a min-max
@@ -342,7 +342,7 @@ select count(*) from functional_parquet.complextypestbl c left outer join
 (select * from c.int_array where item > 10) v;
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=26.00MB mem-reservation=8.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -376,7 +376,7 @@ PLAN-ROOT SINK
      table: rows=unavailable size=unavailable
      columns: unavailable
    extrapolated-rows=disabled
-   mem-estimate=16.00MB mem-reservation=0B
+   mem-estimate=16.00MB mem-reservation=8.00KB
    tuple-ids=0 row-size=16B cardinality=unavailable
 ====
 # Multiple nested collection values (at the same nesting level) where dictionary
@@ -388,7 +388,7 @@ l.l_receiptdate = '1994-08-24' and l.l_shipmode = 'RAIL' and l.l_returnflag = 'R
 l.l_comment is null;
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=176.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=616.00MB mem-reservation=56.00MB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -439,6 +439,6 @@ PLAN-ROOT SINK
    extrapolated-rows=disabled
    parquet statistics predicates on l: l.l_shipdate = '1994-08-19', l.l_receiptdate = '1994-08-24', l.l_shipmode = 'RAIL', l.l_returnflag = 'R'
    parquet dictionary predicates on l: l.l_shipdate = '1994-08-19', l.l_receiptdate = '1994-08-24', l.l_shipmode = 'RAIL', l.l_returnflag = 'R'
-   mem-estimate=176.00MB mem-reservation=0B
+   mem-estimate=616.00MB mem-reservation=56.00MB
    tuple-ids=0 row-size=50B cardinality=150000
 ====

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test b/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
index 610136d..5cbba7b 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/partition-pruning.test
@@ -4,7 +4,7 @@ select * from functional.stringpartitionkey
 where string_col=cast("2009-01-01 00:00:00" as timestamp);
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
-|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=0B
+|  Per-Host Resources: mem-estimate=32.00MB mem-reservation=8.00KB
 PLAN-ROOT SINK
 |  mem-estimate=0B mem-reservation=0B
 |
@@ -15,6 +15,6 @@ PLAN-ROOT SINK
      partitions: 1/1 rows=1
      columns: all
    extrapolated-rows=disabled
-   mem-estimate=32.00MB mem-reservation=0B
+   mem-estimate=32.00MB mem-reservation=8.00KB
    tuple-ids=0 row-size=20B cardinality=1
 ====


[12/15] impala git commit: IMPALA-5752: Add support for DECIMAL on Kudu tables

Posted by ta...@apache.org.
IMPALA-5752: Add support for DECIMAL on Kudu tables

Adds support for the Kudu DECIMAL type introduced in Kudu 1.7.0.

Note: Adding support for Kudu decimal min/max filters is
tracked in IMPALA-6533.

Tests:
* Added Kudu create with decimal test to AnalyzeDDLTest.java
* Added Kudu table_format to test_decimal_queries.py
** Both decimal.test and decimal-exprs.test workloads
* Added decimal queries to the following Kudu workloads:
** kudu_create.test
** kudu_delete.test
** kudu_insert.test
** kudu_update.test
** kudu_upsert.test

Change-Id: I3a9fe5acadc53ec198585d765a8cfb0abe56e199
Reviewed-on: http://gerrit.cloudera.org:8080/9368
Reviewed-by: Dimitris Tsirogiannis <dt...@cloudera.com>
Tested-by: Impala Public Jenkins
Reviewed-on: http://gerrit.cloudera.org:8080/9423
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Tim Armstrong <ta...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/84fffd46
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/84fffd46
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/84fffd46

Branch: refs/heads/2.x
Commit: 84fffd466cf7f7340cd4bcf5639cd2f0a4c559e2
Parents: 07d9f71
Author: Grant Henke <gh...@cloudera.com>
Authored: Tue Feb 20 16:17:40 2018 -0600
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Fri Feb 23 22:51:01 2018 +0000

----------------------------------------------------------------------
 be/src/exec/kudu-scanner.cc                     |   6 +-
 be/src/exec/kudu-table-sink.cc                  |  10 +-
 be/src/exec/kudu-util.cc                        |  71 +++-
 be/src/exec/kudu-util.h                         |  27 +-
 be/src/exprs/kudu-partition-expr.cc             |   4 +-
 .../org/apache/impala/catalog/KuduColumn.java   |   2 +-
 .../org/apache/impala/planner/KuduScanNode.java |   3 +-
 .../impala/planner/RuntimeFilterGenerator.java  |   2 +
 .../impala/service/KuduCatalogOpExecutor.java   |  17 +-
 .../java/org/apache/impala/util/KuduUtil.java   |  43 +-
 .../apache/impala/analysis/AnalyzeDDLTest.java  |  19 +-
 .../functional/functional_schema_template.sql   |  31 ++
 .../datasets/functional/schema_constraints.csv  |   4 +
 .../queries/QueryTest/kudu_create.test          |  36 +-
 .../queries/QueryTest/kudu_delete.test          | 300 ++++++++------
 .../queries/QueryTest/kudu_describe.test        |  20 +
 .../queries/QueryTest/kudu_insert.test          | 255 ++++++------
 .../queries/QueryTest/kudu_update.test          | 270 +++++++------
 .../queries/QueryTest/kudu_upsert.test          | 392 ++++++++++---------
 tests/query_test/test_decimal_queries.py        |   4 +-
 tests/query_test/test_kudu.py                   |   3 +-
 21 files changed, 882 insertions(+), 637 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/be/src/exec/kudu-scanner.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-scanner.cc b/be/src/exec/kudu-scanner.cc
index 909567c..3bc4441 100644
--- a/be/src/exec/kudu-scanner.cc
+++ b/be/src/exec/kudu-scanner.cc
@@ -185,7 +185,7 @@ Status KuduScanner::OpenNextScanToken(const string& scan_token, bool* eos) {
               ctx.filter->filter_desc().targets[it->second];
           const string& col_name = target_desc.kudu_col_name;
           DCHECK(col_name != "");
-          ColumnType col_type = ColumnType::FromThrift(target_desc.kudu_col_type);
+          const ColumnType& col_type = ColumnType::FromThrift(target_desc.kudu_col_type);
 
           void* min = filter->GetMin();
           void* max = filter->GetMax();
@@ -209,14 +209,14 @@ Status KuduScanner::OpenNextScanToken(const string& scan_token, bool* eos) {
           }
 
           KuduValue* min_value;
-          RETURN_IF_ERROR(CreateKuduValue(filter->type(), min, &min_value));
+          RETURN_IF_ERROR(CreateKuduValue(col_type, min, &min_value));
           KUDU_RETURN_IF_ERROR(
               scanner_->AddConjunctPredicate(scan_node_->table_->NewComparisonPredicate(
                   col_name, KuduPredicate::ComparisonOp::GREATER_EQUAL, min_value)),
               "Failed to add min predicate");
 
           KuduValue* max_value;
-          RETURN_IF_ERROR(CreateKuduValue(filter->type(), max, &max_value));
+          RETURN_IF_ERROR(CreateKuduValue(col_type, max, &max_value));
           KUDU_RETURN_IF_ERROR(
               scanner_->AddConjunctPredicate(scan_node_->table_->NewComparisonPredicate(
                   col_name, KuduPredicate::ComparisonOp::LESS_EQUAL, max_value)),

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/be/src/exec/kudu-table-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-table-sink.cc b/be/src/exec/kudu-table-sink.cc
index deb3b66..05f1d06 100644
--- a/be/src/exec/kudu-table-sink.cc
+++ b/be/src/exec/kudu-table-sink.cc
@@ -145,7 +145,9 @@ Status KuduTableSink::Open(RuntimeState* state) {
       return Status(strings::Substitute(
           "Table $0 has fewer columns than expected.", table_desc_->name()));
     }
-    ColumnType type = KuduDataTypeToColumnType(table_->schema().Column(col_idx).type());
+    const KuduColumnSchema& kudu_col = table_->schema().Column(col_idx);
+    const ColumnType& type =
+        KuduDataTypeToColumnType(kudu_col.type(), kudu_col.type_attributes());
     if (type != output_expr_evals_[i]->root().type()) {
       return Status(strings::Substitute("Column $0 has unexpected type. ($1 vs. $2)",
           table_->schema().Column(col_idx).name(), type.DebugString(),
@@ -256,13 +258,13 @@ Status KuduTableSink::Send(RuntimeState* state, RowBatch* batch) {
         }
       }
 
-      PrimitiveType type = output_expr_evals_[j]->root().type().type;
+      const ColumnType& type = output_expr_evals_[j]->root().type();
       Status s = WriteKuduValue(col, type, value, true, write->mutable_row());
       // This can only fail if we set a col to an incorrect type, which would be a bug in
       // planning, so we can DCHECK.
       DCHECK(s.ok()) << "WriteKuduValue failed for col = "
-                     << table_schema.Column(col).name() << " and type = "
-                     << output_expr_evals_[j]->root().type() << ": " << s.GetDetail();
+                     << table_schema.Column(col).name() << " and type = " << type << ": "
+                     << s.GetDetail();
       RETURN_IF_ERROR(s);
     }
     if (add_row) write_ops.push_back(move(write));

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/be/src/exec/kudu-util.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-util.cc b/be/src/exec/kudu-util.cc
index d32df25..0538a61 100644
--- a/be/src/exec/kudu-util.cc
+++ b/be/src/exec/kudu-util.cc
@@ -28,6 +28,7 @@
 #include "common/logging.h"
 #include "common/names.h"
 #include "common/status.h"
+#include "runtime/decimal-value.h"
 #include "runtime/timestamp-value.h"
 #include "runtime/timestamp-value.inline.h"
 
@@ -35,6 +36,7 @@ using kudu::client::KuduSchema;
 using kudu::client::KuduClient;
 using kudu::client::KuduClientBuilder;
 using kudu::client::KuduColumnSchema;
+using kudu::client::KuduColumnTypeAttributes;
 using kudu::client::KuduValue;
 using DataType = kudu::client::KuduColumnSchema::DataType;
 
@@ -71,15 +73,6 @@ Status CreateKuduClient(const vector<string>& master_addrs,
   return Status::OK();
 }
 
-string KuduSchemaDebugString(const KuduSchema& schema) {
-  stringstream ss;
-  for (int i = 0; i < schema.num_columns(); ++i) {
-    const KuduColumnSchema& col = schema.Column(i);
-    ss << col.name() << " " << KuduColumnSchema::DataTypeToString(col.type()) << "\n";
-  }
-  return ss.str();
-}
-
 void LogKuduMessage(void* unused, kudu::client::KuduLogSeverity severity,
     const char* filename, int line_number, const struct ::tm* time, const char* message,
     size_t message_len) {
@@ -123,9 +116,10 @@ static Status ConvertTimestampValue(const TimestampValue* tv, int64_t* ts_micros
   return Status::OK();
 }
 
-Status WriteKuduValue(int col, PrimitiveType type, const void* value,
+Status WriteKuduValue(int col, const ColumnType& col_type, const void* value,
     bool copy_strings, kudu::KuduPartialRow* row) {
-  // TODO: codegen this to eliminate braching on type.
+  // TODO: codegen this to eliminate branching on type.
+  PrimitiveType type = col_type.type;
   switch (type) {
     case TYPE_VARCHAR:
     case TYPE_STRING: {
@@ -172,7 +166,31 @@ Status WriteKuduValue(int col, PrimitiveType type, const void* value,
       RETURN_IF_ERROR(ConvertTimestampValue(
           reinterpret_cast<const TimestampValue*>(value), &ts_micros));
       KUDU_RETURN_IF_ERROR(
-          row->SetUnixTimeMicros(col, ts_micros), "Could not add Kudu WriteOp.");
+          row->SetUnixTimeMicros(col, ts_micros), "Could not set Kudu row value.");
+      break;
+    case TYPE_DECIMAL:
+      switch (col_type.GetByteSize()) {
+        case 4:
+          KUDU_RETURN_IF_ERROR(
+              row->SetUnscaledDecimal(
+                  col, reinterpret_cast<const Decimal4Value*>(value)->value()),
+              "Could not set Kudu row value.");
+          break;
+        case 8:
+          KUDU_RETURN_IF_ERROR(
+              row->SetUnscaledDecimal(
+                  col, reinterpret_cast<const Decimal8Value*>(value)->value()),
+              "Could not set Kudu row value.");
+          break;
+        case 16:
+          KUDU_RETURN_IF_ERROR(
+              row->SetUnscaledDecimal(
+                  col, reinterpret_cast<const Decimal16Value*>(value)->value()),
+              "Could not set Kudu row value.");
+          break;
+        default:
+          DCHECK(false) << "Unknown decimal byte size: " << col_type.GetByteSize();
+      }
       break;
     default:
       return Status(TErrorCode::IMPALA_KUDU_TYPE_MISSING, TypeToString(type));
@@ -181,7 +199,8 @@ Status WriteKuduValue(int col, PrimitiveType type, const void* value,
   return Status::OK();
 }
 
-ColumnType KuduDataTypeToColumnType(DataType type) {
+ColumnType KuduDataTypeToColumnType(
+    DataType type, const KuduColumnTypeAttributes& type_attributes) {
   switch (type) {
     case DataType::INT8: return ColumnType(PrimitiveType::TYPE_TINYINT);
     case DataType::INT16: return ColumnType(PrimitiveType::TYPE_SMALLINT);
@@ -194,13 +213,14 @@ ColumnType KuduDataTypeToColumnType(DataType type) {
     case DataType::BINARY: return ColumnType(PrimitiveType::TYPE_BINARY);
     case DataType::UNIXTIME_MICROS: return ColumnType(PrimitiveType::TYPE_TIMESTAMP);
     case DataType::DECIMAL:
-      DCHECK(false) << "DECIMAL is not supported on Kudu.";
-      return ColumnType(PrimitiveType::INVALID_TYPE);
+      return ColumnType::CreateDecimalType(
+          type_attributes.precision(), type_attributes.scale());
   }
   return ColumnType(PrimitiveType::INVALID_TYPE);
 }
 
-Status CreateKuduValue(PrimitiveType type, void* value, KuduValue** out) {
+Status CreateKuduValue(const ColumnType& col_type, void* value, KuduValue** out) {
+  PrimitiveType type = col_type.type;
   switch (type) {
     case TYPE_VARCHAR:
     case TYPE_STRING: {
@@ -237,6 +257,25 @@ Status CreateKuduValue(PrimitiveType type, void* value, KuduValue** out) {
       *out = KuduValue::FromInt(ts_micros);
       break;
     }
+    case TYPE_DECIMAL: {
+      switch (col_type.GetByteSize()) {
+        case 4:
+          *out = KuduValue::FromDecimal(
+              reinterpret_cast<const Decimal4Value*>(value)->value(), col_type.scale);
+          break;
+        case 8:
+          *out = KuduValue::FromDecimal(
+              reinterpret_cast<const Decimal8Value*>(value)->value(), col_type.scale);
+          break;
+        case 16:
+          *out = KuduValue::FromDecimal(
+              reinterpret_cast<const Decimal16Value*>(value)->value(), col_type.scale);
+          break;
+        default:
+          DCHECK(false) << "Unknown decimal byte size: " << col_type.GetByteSize();
+      }
+      break;
+    }
     default:
       return Status(TErrorCode::IMPALA_KUDU_TYPE_MISSING, TypeToString(type));
   }

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/be/src/exec/kudu-util.h
----------------------------------------------------------------------
diff --git a/be/src/exec/kudu-util.h b/be/src/exec/kudu-util.h
index 5fd1140..36764d4 100644
--- a/be/src/exec/kudu-util.h
+++ b/be/src/exec/kudu-util.h
@@ -65,9 +65,6 @@ bool KuduIsAvailable();
 Status CreateKuduClient(const std::vector<std::string>& master_addrs,
     kudu::client::sp::shared_ptr<kudu::client::KuduClient>* client) WARN_UNUSED_RESULT;
 
-/// Returns a debug string for the KuduSchema.
-std::string KuduSchemaDebugString(const kudu::client::KuduSchema& schema);
-
 /// Initializes Kudu's logging by binding a callback that logs back to Impala's glog. This
 /// also sets Kudu's verbose logging to whatever level is set in Impala.
 void InitKuduLogging();
@@ -78,20 +75,22 @@ void InitKuduLogging();
 void LogKuduMessage(kudu::client::KuduLogSeverity severity, const char* filename,
     int line_number, const struct ::tm* time, const char* message, size_t message_len);
 
-/// Casts 'value' according to 'type' and writes it into 'row' at position 'col'.
-/// If 'type' is STRING or VARCHAR, 'copy_strings' determines if 'value' will be copied
-/// into memory owned by the row. If false, string data must remain valid while the row
-/// is being used.
-Status WriteKuduValue(int col, PrimitiveType type, const void* value,
+/// Casts 'value' according to the column type in 'col_type' and writes it into 'row'
+/// at position 'col'. If the column type's primitive type is STRING or VARCHAR,
+/// 'copy_strings' determines if 'value' will be copied into memory owned by the row.
+/// If false, string data must remain valid while the row is being used.
+Status WriteKuduValue(int col, const ColumnType& col_type, const void* value,
     bool copy_strings, kudu::KuduPartialRow* row) WARN_UNUSED_RESULT;
 
-/// Casts 'value' according to 'type' and create a new KuduValue containing 'value' which
-/// is returned in 'out'.
-Status CreateKuduValue(
-    PrimitiveType type, void* value, kudu::client::KuduValue** out) WARN_UNUSED_RESULT;
+/// Casts 'value' according to the column type in 'col_type' and create a
+/// new KuduValue containing 'value' which is returned in 'out'.
+Status CreateKuduValue(const ColumnType& col_type, void* value,
+    kudu::client::KuduValue** out) WARN_UNUSED_RESULT;
 
-/// Takes a Kudu client DataType and returns the corresponding Impala ColumnType.
-ColumnType KuduDataTypeToColumnType(kudu::client::KuduColumnSchema::DataType type);
+/// Takes a Kudu client DataType and KuduColumnTypeAttributes and
+/// returns the corresponding Impala ColumnType.
+ColumnType KuduDataTypeToColumnType(kudu::client::KuduColumnSchema::DataType type,
+    const kudu::client::KuduColumnTypeAttributes& type_attributes);
 
 /// Utility function for creating an Impala Status object based on a kudu::Status object.
 /// 'k_status' is the kudu::Status object.

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/be/src/exprs/kudu-partition-expr.cc
----------------------------------------------------------------------
diff --git a/be/src/exprs/kudu-partition-expr.cc b/be/src/exprs/kudu-partition-expr.cc
index 2cc96f0..ce38e24 100644
--- a/be/src/exprs/kudu-partition-expr.cc
+++ b/be/src/exprs/kudu-partition-expr.cc
@@ -70,8 +70,8 @@ IntVal KuduPartitionExpr::GetIntVal(ScalarExprEvaluator* eval,
     }
     int col = tkudu_partition_expr_.referenced_columns[i];
     const ColumnDescriptor& col_desc = table_desc_->col_descs()[col];
-    PrimitiveType type = col_desc.type().type;
-    DCHECK_EQ(GetChild(i)->type().type, type);
+    const ColumnType& type = col_desc.type();
+    DCHECK_EQ(GetChild(i)->type().type, type.type);
     Status s = WriteKuduValue(col, type, val, false, row_.get());
     // This can only fail if we set a col to an incorect type, which would be a bug in
     // planning, so we can DCHECK.

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java b/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java
index 2dd3e85..a4b8e5f 100644
--- a/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java
+++ b/fe/src/main/java/org/apache/impala/catalog/KuduColumn.java
@@ -72,7 +72,7 @@ public class KuduColumn extends Column {
 
   public static KuduColumn fromColumnSchema(ColumnSchema colSchema, int position)
       throws ImpalaRuntimeException {
-    Type type = KuduUtil.toImpalaType(colSchema.getType());
+    Type type = KuduUtil.toImpalaType(colSchema.getType(), colSchema.getTypeAttributes());
     Object defaultValue = colSchema.getDefaultValue();
     LiteralExpr defaultValueExpr = null;
     if (defaultValue != null) {

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
index 390592e..a02b49b 100644
--- a/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
+++ b/fe/src/main/java/org/apache/impala/planner/KuduScanNode.java
@@ -162,7 +162,8 @@ public class KuduScanNode extends ScanNode {
             "outdated and need to be refreshed.");
       }
 
-      Type kuduColType = KuduUtil.toImpalaType(kuduCol.getType());
+      Type kuduColType =
+          KuduUtil.toImpalaType(kuduCol.getType(), kuduCol.getTypeAttributes());
       if (!colType.equals(kuduColType)) {
         throw new ImpalaRuntimeException("Column '" + colName + "' is type " +
             kuduColType.toSql() + " but Impala expected " + colType.toSql() +

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java b/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
index 5368b5f..89f14d1 100644
--- a/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
+++ b/fe/src/main/java/org/apache/impala/planner/RuntimeFilterGenerator.java
@@ -699,6 +699,8 @@ public final class RuntimeFilterGenerator {
         continue;
       } else if (scanNode instanceof KuduScanNode) {
         if (filter.getType() != TRuntimeFilterType.MIN_MAX) continue;
+        // TODO: IMPALA-6533: Support Kudu Decimal Min/Max Filters
+        if (targetExpr.getType().isDecimal()) continue;
         SlotRef slotRef = targetExpr.unwrapSlotRef(true);
         // Kudu only supports targeting a single column, not general exprs, so the target
         // must be a SlotRef pointing to a column. We can allow implicit integer casts

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
index c81aca4..0ce0cf9 100644
--- a/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/KuduCatalogOpExecutor.java
@@ -48,6 +48,7 @@ import org.apache.kudu.client.KuduClient;
 import org.apache.kudu.client.KuduException;
 import org.apache.kudu.client.PartialRow;
 import org.apache.kudu.client.RangePartitionBound;
+import org.apache.kudu.util.DecimalUtil;
 import org.apache.log4j.Logger;
 
 import com.google.common.base.Preconditions;
@@ -113,8 +114,8 @@ public class KuduCatalogOpExecutor {
       csb.nullable(!isKey);
     }
     if (column.isSetDefault_value()) {
-      csb.defaultValue(KuduUtil.getKuduDefaultValue(column.getDefault_value(), kuduType,
-            column.getColumnName()));
+      csb.defaultValue(KuduUtil.getKuduDefaultValue(
+          column.getDefault_value(), type, column.getColumnName()));
     }
     if (column.isSetBlock_size()) csb.desiredBlockSize(column.getBlock_size());
     if (column.isSetEncoding()) {
@@ -123,6 +124,10 @@ public class KuduCatalogOpExecutor {
     if (column.isSetCompression()) {
       csb.compressionAlgorithm(KuduUtil.fromThrift(column.getCompression()));
     }
+    if (type.isDecimal()) {
+      csb.typeAttributes(
+          DecimalUtil.typeAttributes(type.getPrecision(), type.getDecimalDigits()));
+    }
     return csb.build();
   }
 
@@ -267,7 +272,8 @@ public class KuduCatalogOpExecutor {
               "Error loading Kudu table: Impala does not support column names that " +
               "differ only in casing '%s'", colSchema.getName()));
         }
-        Type type = KuduUtil.toImpalaType(colSchema.getType());
+        Type type =
+            KuduUtil.toImpalaType(colSchema.getType(), colSchema.getTypeAttributes());
         cols.add(new FieldSchema(colSchema.getName(), type.toSql().toLowerCase(), null));
       }
     } catch (Exception e) {
@@ -446,10 +452,9 @@ public class KuduCatalogOpExecutor {
     AlterTableOptions alterTableOptions = new AlterTableOptions();
 
     if (newCol.isSetDefault_value()) {
-      org.apache.kudu.Type kuduType =
-          KuduUtil.fromImpalaType(Type.fromThrift(newCol.getColumnType()));
+      Type type = Type.fromThrift(newCol.getColumnType());
       Object defaultValue = KuduUtil.getKuduDefaultValue(
-          newCol.getDefault_value(), kuduType, newCol.getColumnName());
+          newCol.getDefault_value(), type, newCol.getColumnName());
       if (defaultValue == null) {
         alterTableOptions.removeDefault(kuduColName);
       } else {

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/fe/src/main/java/org/apache/impala/util/KuduUtil.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/util/KuduUtil.java b/fe/src/main/java/org/apache/impala/util/KuduUtil.java
index 4df8005..07378a9 100644
--- a/fe/src/main/java/org/apache/impala/util/KuduUtil.java
+++ b/fe/src/main/java/org/apache/impala/util/KuduUtil.java
@@ -19,6 +19,8 @@ package org.apache.impala.util;
 
 import static java.lang.String.format;
 
+import java.math.BigDecimal;
+import java.math.BigInteger;
 import java.util.List;
 import java.util.Map;
 
@@ -48,6 +50,7 @@ import org.apache.impala.thrift.THdfsCompression;
 import org.apache.kudu.ColumnSchema;
 import org.apache.kudu.ColumnSchema.CompressionAlgorithm;
 import org.apache.kudu.ColumnSchema.Encoding;
+import org.apache.kudu.ColumnTypeAttributes;
 import org.apache.kudu.Schema;
 import org.apache.kudu.client.KuduClient;
 import org.apache.kudu.client.KuduClient.KuduClientBuilder;
@@ -104,8 +107,7 @@ public class KuduUtil {
       String colName = rangePartitionColumns.get(i);
       ColumnSchema col = schema.getColumn(colName);
       Preconditions.checkNotNull(col);
-      setKey(col.getType(), boundaryValues.get(i), schema.getColumnIndex(colName),
-          colName, bound);
+      setKey(col, boundaryValues.get(i), schema.getColumnIndex(colName), bound);
     }
     return bound;
   }
@@ -140,10 +142,12 @@ public class KuduUtil {
    * Sets the value 'boundaryVal' in 'key' at 'pos'. Checks if 'boundaryVal' has the
    * expected data type.
    */
-  private static void setKey(org.apache.kudu.Type type, TExpr boundaryVal, int pos,
-      String colName, PartialRow key) throws ImpalaRuntimeException {
+  private static void setKey(ColumnSchema col, TExpr boundaryVal, int pos, PartialRow key)
+      throws ImpalaRuntimeException {
     Preconditions.checkState(boundaryVal.getNodes().size() == 1);
     TExprNode literal = boundaryVal.getNodes().get(0);
+    String colName = col.getName();
+    org.apache.kudu.Type type = col.getType();
     switch (type) {
       case INT8:
         checkCorrectType(literal.isSetInt_literal(), type, colName, literal);
@@ -169,6 +173,12 @@ public class KuduUtil {
         checkCorrectType(literal.isSetInt_literal(), type, colName, literal);
         key.addLong(pos, literal.getInt_literal().getValue());
         break;
+      case DECIMAL:
+        checkCorrectType(literal.isSetDecimal_literal(), type, colName, literal);
+        BigInteger unscaledVal = new BigInteger(literal.getDecimal_literal().getValue());
+        int scale = col.getTypeAttributes().getScale();
+        key.addDecimal(pos, new BigDecimal(unscaledVal, scale));
+        break;
       default:
         throw new ImpalaRuntimeException("Key columns not supported for type: "
             + type.toString());
@@ -177,15 +187,17 @@ public class KuduUtil {
 
   /**
    * Returns the actual value of the specified defaultValue literal. The returned type is
-   * the value type stored by Kudu for the column. E.g. if 'type' is 'INT8', the returned
-   * value is a Java byte, and if 'type' is 'UNIXTIME_MICROS', the returned value is
+   * the value type stored by Kudu for the column. For example, The `impalaType` is
+   * translated to a Kudu Type 'type' and if 'type' is 'INT8', the returned
+   * value is a Java byte, or if 'type' is 'UNIXTIME_MICROS', the returned value is
    * a Java long.
    */
-  public static Object getKuduDefaultValue(TExpr defaultValue,
-      org.apache.kudu.Type type, String colName) throws ImpalaRuntimeException {
+  public static Object getKuduDefaultValue(
+      TExpr defaultValue, Type impalaType, String colName) throws ImpalaRuntimeException {
     Preconditions.checkState(defaultValue.getNodes().size() == 1);
     TExprNode literal = defaultValue.getNodes().get(0);
     if (literal.getNode_type() == TExprNodeType.NULL_LITERAL) return null;
+    org.apache.kudu.Type type = KuduUtil.fromImpalaType(impalaType);
     switch (type) {
       case INT8:
         checkCorrectType(literal.isSetInt_literal(), type, colName, literal);
@@ -214,6 +226,10 @@ public class KuduUtil {
       case UNIXTIME_MICROS:
         checkCorrectType(literal.isSetInt_literal(), type, colName, literal);
         return literal.getInt_literal().getValue();
+      case DECIMAL:
+        checkCorrectType(literal.isSetDecimal_literal(), type, colName, literal);
+        BigInteger unscaledVal = new BigInteger(literal.getDecimal_literal().getValue());
+        return new BigDecimal(unscaledVal, impalaType.getDecimalDigits());
       default:
         throw new ImpalaRuntimeException("Unsupported value for column type: " +
             type.toString());
@@ -392,13 +408,13 @@ public class KuduUtil {
       case DOUBLE: return org.apache.kudu.Type.DOUBLE;
       case FLOAT: return org.apache.kudu.Type.FLOAT;
       case TIMESTAMP: return org.apache.kudu.Type.UNIXTIME_MICROS;
-        /* Fall through below */
+      case DECIMAL: return org.apache.kudu.Type.DECIMAL;
+      /* Fall through below */
       case INVALID_TYPE:
       case NULL_TYPE:
       case BINARY:
       case DATE:
       case DATETIME:
-      case DECIMAL:
       case CHAR:
       case VARCHAR:
       default:
@@ -407,8 +423,8 @@ public class KuduUtil {
     }
   }
 
-  public static Type toImpalaType(org.apache.kudu.Type t)
-      throws ImpalaRuntimeException {
+  public static Type toImpalaType(org.apache.kudu.Type t,
+      ColumnTypeAttributes typeAttributes) throws ImpalaRuntimeException {
     switch (t) {
       case BOOL: return Type.BOOLEAN;
       case DOUBLE: return Type.DOUBLE;
@@ -419,6 +435,9 @@ public class KuduUtil {
       case INT64: return Type.BIGINT;
       case STRING: return Type.STRING;
       case UNIXTIME_MICROS: return Type.TIMESTAMP;
+      case DECIMAL:
+        return ScalarType.createDecimalType(
+            typeAttributes.getPrecision(), typeAttributes.getScale());
       default:
         throw new ImpalaRuntimeException(String.format(
             "Kudu type '%s' is not supported in Impala", t.getName()));

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
----------------------------------------------------------------------
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 56f81e3..ce49828 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -1792,6 +1792,10 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "left join functional.alltypes b " +
         "on b.timestamp_col between a.timestamp_col and a.timestamp_col) " +
         "select a.timestamp_col, a.year from tmp a");
+    // CTAS into Kudu with decimal type
+    AnalyzesOk("create table t primary key (id) partition by hash partitions 3" +
+        " stored as kudu as select c1 as id from functional.decimal_tiny");
+
     // CTAS in an external Kudu table
     AnalysisError("create external table t stored as kudu " +
         "tblproperties('kudu.table_name'='t') as select id, int_col from " +
@@ -1806,9 +1810,6 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         " stored as kudu as select vc from functional.chars_tiny",
         "Cannot create table 't': Type VARCHAR(32) is not supported in Kudu");
     AnalysisError("create table t primary key (id) partition by hash partitions 3" +
-        " stored as kudu as select c1 as id from functional.decimal_tiny",
-        "Cannot create table 't': Type DECIMAL(10,4) is not supported in Kudu");
-    AnalysisError("create table t primary key (id) partition by hash partitions 3" +
         " stored as kudu as select id, s from functional.complextypes_fileformat",
         "Expr 's' in select list returns a complex type 'STRUCT<f1:STRING,f2:INT>'.\n" +
         "Only scalar types are allowed in the select list.");
@@ -2212,6 +2213,9 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalyzesOk("alter table functional_kudu.testtbl add columns (a1 tinyint null, a2 " +
         "smallint null, a3 int null, a4 bigint null, a5 string null, a6 float null, " +
         "a7 double null, a8 boolean null comment 'boolean')");
+    // Decimal types
+    AnalyzesOk("alter table functional_kudu.testtbl add columns (d1 decimal null, d2 " +
+        "decimal(9, 2) null, d3 decimal(15, 15) null, d4 decimal(38, 0) null)");
     // Complex types
     AnalysisError("alter table functional_kudu.testtbl add columns ( "+
         "a struct<f1:int>)", "Kudu tables do not support complex types: " +
@@ -2225,6 +2229,8 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "default 10)");
     AnalyzesOk("alter table functional_kudu.testtbl add columns (a1 int null " +
         "default 10)");
+    AnalyzesOk("alter table functional_kudu.testtbl add columns (d1 decimal(9, 2) null " +
+        "default 99.99)");
     // Other Kudu column options
     AnalyzesOk("alter table functional_kudu.testtbl add columns (a int encoding rle)");
     AnalyzesOk("alter table functional_kudu.testtbl add columns (a int compression lz4)");
@@ -2477,8 +2483,7 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "in Kudu tables.");
 
     // Test unsupported Kudu types
-    List<String> unsupportedTypes = Lists.newArrayList(
-        "DECIMAL(9,0)", "VARCHAR(20)", "CHAR(20)",
+    List<String> unsupportedTypes = Lists.newArrayList("VARCHAR(20)", "CHAR(20)",
         "STRUCT<f1:INT,f2:STRING>", "ARRAY<INT>", "MAP<STRING,STRING>");
     for (String t: unsupportedTypes) {
       String expectedError = String.format(
@@ -2529,7 +2534,8 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalyzesOk("create table tab (x int primary key, i1 tinyint default null, " +
         "i2 smallint default null, i3 int default null, i4 bigint default null, " +
         "vals string default null, valf float default null, vald double default null, " +
-        "valb boolean default null) partition by hash (x) partitions 3 stored as kudu");
+        "valb boolean default null, valdec decimal(10, 5) default null) " +
+        "partition by hash (x) partitions 3 stored as kudu");
     // Use NULL as a default value on a non-nullable column
     AnalysisError("create table tab (x int primary key, y int not null default null) " +
         "partition by hash (x) partitions 3 stored as kudu", "Default value of NULL " +
@@ -2561,6 +2567,7 @@ public class AnalyzeDDLTest extends FrontendTestBase {
         "i3 int default 100, i4 bigint default 1000, vals string default 'test', " +
         "valf float default cast(1.2 as float), vald double default " +
         "cast(3.1452 as double), valb boolean default true, " +
+        "valdec decimal(10, 5) default 3.14159, " +
         "primary key (i1, i2, i3, i4, vals)) partition by hash (i1) partitions 3 " +
         "stored as kudu");
     AnalyzesOk("create table tab (i int primary key default 1+1+1) " +

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/testdata/datasets/functional/functional_schema_template.sql
----------------------------------------------------------------------
diff --git a/testdata/datasets/functional/functional_schema_template.sql b/testdata/datasets/functional/functional_schema_template.sql
index f4b6b34..8db806a 100644
--- a/testdata/datasets/functional/functional_schema_template.sql
+++ b/testdata/datasets/functional/functional_schema_template.sql
@@ -1726,6 +1726,23 @@ ${IMPALA_HOME}/testdata/data/decimal_tbl.txt /test-warehouse/decimal_tbl/d6=1/
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name} partition(d6)
 select * from functional.{table_name};
+---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
+CREATE TABLE {db_name}{db_suffix}.{table_name} (
+  d1 DECIMAL,
+  d2 DECIMAL(10, 0),
+  d3 DECIMAL(20, 10),
+  d4 DECIMAL(38, 38),
+  d5 DECIMAL(10, 5),
+  d6 DECIMAL(9, 0),
+  PRIMARY KEY (d1, d2, d3, d4, d5, d6)
+)
+PARTITION BY HASH PARTITIONS 3
+STORED AS KUDU;
+---- DEPENDENT_LOAD_KUDU
+INSERT into TABLE {db_name}{db_suffix}.{table_name}
+SELECT d1, d2, d3, d4, d5, d6
+FROM {db_name}.{table_name};
 ====
 ---- DATASET
 functional
@@ -1743,6 +1760,20 @@ ${IMPALA_HOME}/testdata/data/decimal-tiny.txt /test-warehouse/decimal_tiny/
 ---- DEPENDENT_LOAD
 INSERT OVERWRITE TABLE {db_name}{db_suffix}.{table_name}
 select * from functional.{table_name};
+---- CREATE_KUDU
+DROP TABLE IF EXISTS {db_name}{db_suffix}.{table_name};
+CREATE TABLE {db_name}{db_suffix}.{table_name} (
+  c1 DECIMAL(10, 4),
+  c2 DECIMAL(15, 5),
+  c3 DECIMAL(1, 1),
+  PRIMARY KEY (c1, c2, c3)
+)
+PARTITION BY HASH PARTITIONS 3
+STORED AS KUDU;
+---- DEPENDENT_LOAD_KUDU
+INSERT into TABLE {db_name}{db_suffix}.{table_name}
+SELECT c1, c2, c3
+FROM {db_name}.{table_name};
 ====
 ---- DATASET
 functional

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/testdata/datasets/functional/schema_constraints.csv
----------------------------------------------------------------------
diff --git a/testdata/datasets/functional/schema_constraints.csv b/testdata/datasets/functional/schema_constraints.csv
index 355e9b1..ef65b9a 100644
--- a/testdata/datasets/functional/schema_constraints.csv
+++ b/testdata/datasets/functional/schema_constraints.csv
@@ -132,6 +132,8 @@ table_name:decimal_tbl, constraint:restrict_to, table_format:text/none/none
 table_name:decimal_tiny, constraint:restrict_to, table_format:text/none/none
 table_name:decimal_tbl, constraint:restrict_to, table_format:parquet/none/none
 table_name:decimal_tiny, constraint:restrict_to, table_format:parquet/none/none
+table_name:decimal_tbl, constraint:restrict_to, table_format:kudu/none/none
+table_name:decimal_tiny, constraint:restrict_to, table_format:kudu/none/none
 
 table_name:avro_decimal_tbl, constraint:restrict_to, table_format:avro/snap/block
 
@@ -186,6 +188,8 @@ table_name:tinyinttable, constraint:only, table_format:kudu/none/none
 table_name:zipcode_incomes, constraint:only, table_format:kudu/none/none
 table_name:nulltable, constraint:only, table_format:kudu/none/none
 table_name:nullescapedtable, constraint:only, table_format:kudu/none/none
+table_name:decimal_tbl, constraint:only, table_format:kudu/none/none
+table_name:decimal_tiny, constraint:only, table_format:kudu/none/none
 
 # Skipping header lines is only effective with text tables
 table_name:table_with_header, constraint:restrict_to, table_format:text/none/none

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
index f6e16e1..24dfdf1 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_create.test
@@ -129,7 +129,9 @@ INT,STRING,FLOAT,BIGINT
 create table tbl_with_null_defaults (x int primary key, i1 tinyint default null,
   i2 smallint default null, i3 int default null, i4 bigint default null,
   vals string default null, valf float default null, vald double default null,
-  valb boolean default null) partition by hash (x) partitions 3 stored as kudu
+  valb boolean default null, valdec4 decimal(9) default null,
+  valdec8 decimal(18) default null, valdec16 decimal(38) default null)
+  partition by hash (x) partitions 3 stored as kudu
 ---- RESULTS
 ====
 ---- QUERY
@@ -138,11 +140,11 @@ insert into tbl_with_null_defaults (x) values (1);
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-X, I1, I2, I3, I4, VALS, VALF, VALD, VALB
+X, I1, I2, I3, I4, VALS, VALF, VALD, VALB, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tbl_with_null_defaults
-1,NULL,NULL,NULL,NULL,'NULL',NULL,NULL,NULL
+1,NULL,NULL,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,TINYINT,SMALLINT,INT,BIGINT,STRING,FLOAT,DOUBLE,BOOLEAN
+INT,TINYINT,SMALLINT,INT,BIGINT,STRING,FLOAT,DOUBLE,BOOLEAN,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # Overlapping ranges are rejected by the Kudu client
@@ -287,4 +289,30 @@ select * from unpartitioned_kudu_table2
 ID
 ---- TYPES
 INT
+====
+---- QUERY
+# Creates a Kudu table with decimal columns and primary key
+create table create_decimal
+(
+ decimal_4 decimal(9, 9),
+ decimal_8 decimal(18, 2) not null default 100.00,
+ decimal_16 decimal(38, 0) null,
+ primary key (decimal_4))
+stored as kudu;
+---- RESULTS
+====
+---- QUERY
+# Create as select table with decimal columns and primary key
+create table ctas_decimal primary key (d1,d2,d3)
+stored as kudu
+as select * from functional.decimal_tbl;
+select * from ctas_decimal;
+---- RESULTS
+1234,2222,1.2345678900,0.12345678900000000000000000000000000000,12345.78900,1
+2345,111,12.3456789000,0.12345678900000000000000000000000000000,3.14100,1
+12345,333,123.4567890000,0.12345678900000000000000000000000000000,11.22000,1
+12345,333,1234.5678900000,0.12345678900000000000000000000000000000,0.10000,1
+132842,333,12345.6789000000,0.12345678900000000000000000000000000000,0.77889,1
+---- TYPES
+DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL,DECIMAL
 ====
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
index 7094c59..3abcdab 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_delete.test
@@ -2,23 +2,25 @@
 ---- QUERY
 create table tdata
   (id int primary key, valf float null, vali bigint null, valv string null,
-   valb boolean null, valt tinyint null, vals smallint null, vald double null)
+   valb boolean null, valt tinyint null, vals smallint null, vald double null,
+   valdec4 decimal(9,9) null, valdec8 decimal(18,2) null,
+   valdec16 decimal(38, 0) null)
   PARTITION BY RANGE (PARTITION VALUES < 100, PARTITION 100 <= VALUES < 1000,
   PARTITION 1000 <= VALUES <= 10000) STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY
 insert into table tdata values
-(1, 1.0, 1, 'one', true, 1, 1, 1),
-(2, -2, 20, 'two', false, 0, 1, NULL),
-(3, 0, NULL, 'three', false, 10, 20, 30),
-(4, 5, 6, 'four', true, 7, 8, 9),
-(5, 0, 10, 'five', NULL, 15, 20, 25),
-(6, 9, 12, 'six', true, -1, -2, cast('inf' as double)),
-(7, NULL, 7, 'seven', false, 77, 777, NULL),
-(8, 0, 80, NULL, true, 10, 11, 12),
-(9, NULL, NULL, NULL, NULL, NULL, NULL, NULL),
-(127, 1, 2, '127', false, 3, 4, 5)
+(1, 1.0, 1, 'one', true, 1, 1, 1, 0.000000001, 1.11, 1),
+(2, -2, 20, 'two', false, 0, 1, NULL, 0.000000002, 2.22, 2),
+(3, 0, NULL, 'three', false, 10, 20, 30, 0.000000040, 50.00, 60),
+(4, 5, 6, 'four', true, 7, 8, 9, 0.000000010, 11.11, 12),
+(5, 0, 10, 'five', NULL, 15, 20, 25, 0.000000030, 35.35, 40),
+(6, 9, 12, 'six', true, -1, -2, cast('inf' as double), -0.000000001, -1.11, -1),
+(7, NULL, 7, 'seven', false, 77, 777, NULL, NULL, NULL, NULL),
+(8, 0, 80, NULL, true, 10, 11, 12, 0.000000013, 0.14, 15),
+(9, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL),
+(127, 1, 2, '127', false, 3, 4, 5, 0.000000000, 0.00, 0)
 ---- RESULTS
 : 10
 ====
@@ -29,19 +31,19 @@ delete from tdata where id = 1
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-4,5,6,'four',true,7,8,9
-5,0,10,'five',NULL,15,20,25
-6,9,12,'six',true,-1,-2,Infinity
-7,NULL,7,'seven',false,77,777,NULL
-8,0,80,'NULL',true,10,11,12
-9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-127,1,2,'127',false,3,4,5
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+4,5,6,'four',true,7,8,9,0.000000010,11.11,12
+5,0,10,'five',NULL,15,20,25,0.000000030,35.35,40
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+7,NULL,7,'seven',false,77,777,NULL,NULL,NULL,NULL
+8,0,80,'NULL',true,10,11,12,0.000000013,0.14,15
+9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL
+127,1,2,'127',false,3,4,5,0.000000000,0.00,0
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # predicate on key, NULL
@@ -50,19 +52,19 @@ delete from tdata where id is NULL
 NumModifiedRows: 0
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-4,5,6,'four',true,7,8,9
-5,0,10,'five',NULL,15,20,25
-6,9,12,'six',true,-1,-2,Infinity
-7,NULL,7,'seven',false,77,777,NULL
-8,0,80,'NULL',true,10,11,12
-9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-127,1,2,'127',false,3,4,5
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+4,5,6,'four',true,7,8,9,0.000000010,11.11,12
+5,0,10,'five',NULL,15,20,25,0.000000030,35.35,40
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+7,NULL,7,'seven',false,77,777,NULL,NULL,NULL,NULL
+8,0,80,'NULL',true,10,11,12,0.000000013,0.14,15
+9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL
+127,1,2,'127',false,3,4,5,0.000000000,0.00,0
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # predicate on key, nothing is deleted
@@ -71,19 +73,19 @@ delete from tdata where id = 10
 NumModifiedRows: 0
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-4,5,6,'four',true,7,8,9
-5,0,10,'five',NULL,15,20,25
-6,9,12,'six',true,-1,-2,Infinity
-7,NULL,7,'seven',false,77,777,NULL
-8,0,80,'NULL',true,10,11,12
-9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-127,1,2,'127',false,3,4,5
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+4,5,6,'four',true,7,8,9,0.000000010,11.11,12
+5,0,10,'five',NULL,15,20,25,0.000000030,35.35,40
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+7,NULL,7,'seven',false,77,777,NULL,NULL,NULL,NULL
+8,0,80,'NULL',true,10,11,12,0.000000013,0.14,15
+9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL
+127,1,2,'127',false,3,4,5,0.000000000,0.00,0
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # predicate on key, boundary value
@@ -92,18 +94,18 @@ delete from tdata where id = max_tinyint()
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-4,5,6,'four',true,7,8,9
-5,0,10,'five',NULL,15,20,25
-6,9,12,'six',true,-1,-2,Infinity
-7,NULL,7,'seven',false,77,777,NULL
-8,0,80,'NULL',true,10,11,12
-9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+4,5,6,'four',true,7,8,9,0.000000010,11.11,12
+5,0,10,'five',NULL,15,20,25,0.000000030,35.35,40
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+7,NULL,7,'seven',false,77,777,NULL,NULL,NULL,NULL
+8,0,80,'NULL',true,10,11,12,0.000000013,0.14,15
+9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # compound predicate on key
@@ -112,17 +114,17 @@ delete from tdata where id > 6 and id < 8
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-4,5,6,'four',true,7,8,9
-5,0,10,'five',NULL,15,20,25
-6,9,12,'six',true,-1,-2,Infinity
-8,0,80,'NULL',true,10,11,12
-9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+4,5,6,'four',true,7,8,9,0.000000010,11.11,12
+5,0,10,'five',NULL,15,20,25,0.000000030,35.35,40
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+8,0,80,'NULL',true,10,11,12,0.000000013,0.14,15
+9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # predicate on key, multiple rows
@@ -131,23 +133,23 @@ delete from tdata where id % 4 = 0
 NumModifiedRows: 2
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-5,0,10,'five',NULL,15,20,25
-6,9,12,'six',true,-1,-2,Infinity
-9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+5,0,10,'five',NULL,15,20,25,0.000000030,35.35,40
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # insert new values, including some that were previously deleted
 insert into table tdata values
-(10, 20, 30, 'ten', true, 40, 50, 60),
-(1, 1.0, 1, 'one', true, 1, 1, 1),
-(11, -11, 11, 'eleven', false, 1, 11, 111),
-(8, 0, 80, NULL, true, 10, 11, 12)
+(10, 20, 30, 'ten', true, 40, 50, 60, 0.000000070, 80.80, 90),
+(1, 1.0, 1, 'one', true, 1, 1, 1, 0.000000001, 1.11, 1),
+(11, -11, 11, 'eleven', false, 1, 11, 111, 0.000000011, 11.00, 11),
+(8, 0, 80, NULL, true, 10, 11, 12, 0.000000013, 0.14, 15)
 ====
 ---- QUERY
 # single row, predicate on non-key
@@ -156,18 +158,18 @@ delete from tdata where valv = 'five'
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-6,9,12,'six',true,-1,-2,Infinity
-9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-10,20,30,'ten',true,40,50,60
-1,1.0,1,'one',true,1,1,1
-11,-11,11,'eleven',false,1,11,111
-8,0,80,'NULL',true,10,11,12
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+9,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL
+10,20,30,'ten',true,40,50,60,0.000000070,80.80,90
+1,1.0,1,'one',true,1,1,1,0.000000001,1.11,1
+11,-11,11,'eleven',false,1,11,111,0.000000011,11.00,11
+8,0,80,'NULL',true,10,11,12,0.000000013,0.14,15
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # predicate on non-key, NULL
@@ -176,17 +178,17 @@ delete from tdata where valb is NULL
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-6,9,12,'six',true,-1,-2,Infinity
-10,20,30,'ten',true,40,50,60
-1,1.0,1,'one',true,1,1,1
-11,-11,11,'eleven',false,1,11,111
-8,0,80,'NULL',true,10,11,12
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+10,20,30,'ten',true,40,50,60,0.000000070,80.80,90
+1,1.0,1,'one',true,1,1,1,0.000000001,1.11,1
+11,-11,11,'eleven',false,1,11,111,0.000000011,11.00,11
+8,0,80,'NULL',true,10,11,12,0.000000013,0.14,15
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # predicate on non-key, nothing is deleted
@@ -195,17 +197,17 @@ delete from tdata where vals = -100
 NumModifiedRows: 0
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-6,9,12,'six',true,-1,-2,Infinity
-10,20,30,'ten',true,40,50,60
-1,1.0,1,'one',true,1,1,1
-11,-11,11,'eleven',false,1,11,111
-8,0,80,'NULL',true,10,11,12
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+10,20,30,'ten',true,40,50,60,0.000000070,80.80,90
+1,1.0,1,'one',true,1,1,1,0.000000001,1.11,1
+11,-11,11,'eleven',false,1,11,111,0.000000011,11.00,11
+8,0,80,'NULL',true,10,11,12,0.000000013,0.14,15
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # predicate on non-key, compound predicate
@@ -214,16 +216,16 @@ delete from tdata where valf = 0 and vali = 80
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-3,0,NULL,'three',false,10,20,30
-6,9,12,'six',true,-1,-2,Infinity
-10,20,30,'ten',true,40,50,60
-1,1.0,1,'one',true,1,1,1
-11,-11,11,'eleven',false,1,11,111
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+10,20,30,'ten',true,40,50,60,0.000000070,80.80,90
+1,1.0,1,'one',true,1,1,1,0.000000001,1.11,1
+11,-11,11,'eleven',false,1,11,111,0.000000011,11.00,11
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # predicate on non-key, multiple rows
@@ -232,14 +234,14 @@ delete from tdata where vals % 10 = 0
 NumModifiedRows: 2
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-6,9,12,'six',true,-1,-2,Infinity
-1,1.0,1,'one',true,1,1,1
-11,-11,11,'eleven',false,1,11,111
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+1,1.0,1,'one',true,1,1,1,0.000000001,1.11,1
+11,-11,11,'eleven',false,1,11,111,0.000000011,11.00,11
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # 'from' syntax - the join results in four deletes, 3 of which fail
@@ -248,13 +250,13 @@ delete a from tdata a, tdata b where a.id = 11
 NumModifiedRows: 1
 NumRowErrors: 3
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-6,9,12,'six',true,-1,-2,Infinity
-1,1.0,1,'one',true,1,1,1
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+1,1.0,1,'one',true,1,1,1,0.000000001,1.11,1
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # Try to delete a row with a primary key value that is not covered by the existing range
@@ -264,18 +266,56 @@ delete from tdata where id = 10001
 NumModifiedRows: 0
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-2,-2,20,'two',false,0,1,NULL
-6,9,12,'six',true,-1,-2,Infinity
-1,1.0,1,'one',true,1,1,1
+2,-2,20,'two',false,0,1,NULL,0.000000002,2.22,2
+6,9,12,'six',true,-1,-2,Infinity,-0.000000001,-1.11,-1
+1,1.0,1,'one',true,1,1,1,0.000000001,1.11,1
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
+====
+---- QUERY
+# Add a couple more rows back
+insert into table tdata values
+(3, 0, NULL, 'three', false, 10, 20, 30, 0.000000040, 50.00, 60),
+(4, 5, 6, 'four', true, 7, 8, 9, 0.000000010, 11.11, 12),
+(5, 0, 10, 'five', NULL, 15, 20, 25, 0.000000030, 35.35, 40)
+---- RESULTS
+: 3
+====
+---- QUERY
+# predicate on decimal, multiple rows
+delete from tdata where valdec8 < 11.11
+---- RUNTIME_PROFILE
+NumModifiedRows: 3
+NumRowErrors: 0
+---- LABELS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
+---- DML_RESULTS: tdata
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+4,5,6,'four',true,7,8,9,0.000000010,11.11,12
+5,0,10,'five',NULL,15,20,25,0.000000030,35.35,40
+---- TYPES
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
+====
+---- QUERY
+# predicate on decimal, single row
+delete from tdata where valdec4 = 0.000000030
+---- RUNTIME_PROFILE
+NumModifiedRows: 1
+NumRowErrors: 0
+---- LABELS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
+---- DML_RESULTS: tdata
+3,0,NULL,'three',false,10,20,30,0.000000040,50.00,60
+4,5,6,'four',true,7,8,9,0.000000010,11.11,12
+---- TYPES
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 insert into tdata
 select cast(id + 100 as int), float_col, bigint_col, string_col, bool_col, tinyint_col,
-smallint_col, double_col
+smallint_col, double_col, NULL, NULL, NULL
 from functional_kudu.alltypes
 ---- RESULTS
 : 7300
@@ -287,13 +327,13 @@ NumRowErrors: 0
 # Test a larger DELETE
 delete from tdata where id > -1
 ---- RUNTIME_PROFILE
-NumModifiedRows: 7303
+NumModifiedRows: 7302
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 create table multiple_key_cols

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test
index 687ca31..daaa18f 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_describe.test
@@ -45,3 +45,23 @@ NAME,TYPE,COMMENT,PRIMARY_KEY,NULLABLE,DEFAULT_VALUE,ENCODING,COMPRESSION,BLOCK_
 ---- TYPES
 STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING
 ====
+---- QUERY
+# Test decimal columns and primary key
+create table describe_decimal_test
+(
+ decimal_default decimal PRIMARY KEY,
+ decimal_4 decimal(9, 9) not null,
+ decimal_8 decimal(18, 2) not null default 100.00,
+ decimal_16 decimal(38, 0) null)
+stored as kudu;
+describe describe_decimal_test;
+---- LABELS
+NAME,TYPE,COMMENT,PRIMARY_KEY,NULLABLE,DEFAULT_VALUE,ENCODING,COMPRESSION,BLOCK_SIZE
+---- RESULTS
+'decimal_default','decimal(9,0)','','true','false','','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+'decimal_4','decimal(9,9)','','false','false','','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+'decimal_8','decimal(18,2)','','false','false','100.00','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+'decimal_16','decimal(38,0)','','false','true','','AUTO_ENCODING','DEFAULT_COMPRESSION','0'
+---- TYPES
+STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING,STRING
+====

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
index 420e42c..6bba77a 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_insert.test
@@ -3,7 +3,8 @@
 create table tdata
   (id int primary key, valf float null, vali bigint null, valv string null,
    valb boolean null, valt tinyint null, vals smallint null, vald double null,
-   ts timestamp)
+   ts timestamp, decimal4 decimal(9,9) null, decimal8 decimal(18,2) null,
+   decimal16 decimal(38, 0) null)
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES) STORED AS KUDU
 ---- RESULTS
@@ -11,30 +12,30 @@ create table tdata
 ---- QUERY
 # VALUES, single row, all target cols, no errors
 insert into tdata values (1, 1, 1, 'one', true, 1, 1, 1,
-  cast('1987-05-19 00:00:00' as timestamp))
+  cast('1987-05-19 00:00:00' as timestamp), 0.000000001, 1.00, 1)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-1,1,1,'one',true,1,1,1,1987-05-19 00:00:00
+1,1,1,'one',true,1,1,1,1987-05-19 00:00:00,0.000000001,1.00,1
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all target cols, NULL
-insert into tdata values (2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)
+insert into tdata values (2, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-1,1,1,'one',true,1,1,1,1987-05-19 00:00:00
-2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+1,1,1,'one',true,1,1,1,1987-05-19 00:00:00,0.000000001,1.00,1
+2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all target cols, boundary values. The timestamp value is the max
@@ -43,18 +44,19 @@ INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
 insert into tdata values
 (3, cast('nan' as float), max_bigint(), '', true, min_tinyint(), max_smallint(),
   cast('-inf' as double),
-  nanoseconds_add(cast('9999-12-31 23:59:59' as timestamp), 999999999))
+  nanoseconds_add(cast('9999-12-31 23:59:59' as timestamp), 999999999),
+  0.999999999, 9999999999999999.99, 99999999999999999999999999999999999999)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-1,1,1,'one',true,1,1,1,1987-05-19 00:00:00
-2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
-3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000
+1,1,1,'one',true,1,1,1,1987-05-19 00:00:00,0.000000001,1.00,1
+2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000,0.999999999,9999999999999999.99,99999999999999999999999999999999999999
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, subset of target cols
@@ -63,36 +65,36 @@ insert into tdata (valb, vald, id) values (true, 0, 4)
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-1,1,1,'one',true,1,1,1,1987-05-19 00:00:00
-2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
-3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000
-4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL
+1,1,1,'one',true,1,1,1,1987-05-19 00:00:00,0.000000001,1.00,1
+2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000,0.999999999,9999999999999999.99,99999999999999999999999999999999999999
+4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # VALUES, multiple rows, all target cols
 insert into tdata values
-(5, 5.0, 5, 'five', false, NULL, NULL, NULL, NULL),
-(6, 16, 60, '', true, 0, -1, -6, cast('2010-12-31 23:59:59' as timestamp)),
-(7, NULL, 10, NULL, false, max_tinyint(), -7, 2, cast('1400-01-01 00:00:00' as timestamp))
+(5, 5.0, 5, 'five', false, NULL, NULL, NULL, NULL, NULL, NULL, NULL),
+(6, 16, 60, '', true, 0, -1, -6, cast('2010-12-31 23:59:59' as timestamp), -0.000000001, -1.00, -1),
+(7, NULL, 10, NULL, false, max_tinyint(), -7, 2, cast('1400-01-01 00:00:00' as timestamp), 0.000000000, 0.00, 0)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 3
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-1,1,1,'one',true,1,1,1,1987-05-19 00:00:00
-2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
-3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000
-4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL
-5,5.0,5,'five',false,NULL,NULL,NULL,NULL
-6,16,60,'',true,0,-1,-6,2010-12-31 23:59:59
-7,NULL,10,'NULL',false,127,-7,2,1400-01-01 00:00:00
+1,1,1,'one',true,1,1,1,1987-05-19 00:00:00,0.000000001,1.00,1
+2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000,0.999999999,9999999999999999.99,99999999999999999999999999999999999999
+4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL,NULL,NULL,NULL
+5,5.0,5,'five',false,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+6,16,60,'',true,0,-1,-6,2010-12-31 23:59:59,-0.000000001,-1.00,-1
+7,NULL,10,'NULL',false,127,-7,2,1400-01-01 00:00:00,0.000000000,0.00,0
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # VALUES, multiple rows, subset of cols
@@ -103,44 +105,44 @@ insert into tdata (valv, valf, vali, id) values
 NumModifiedRows: 2
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-1,1,1,'one',true,1,1,1,1987-05-19 00:00:00
-2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
-3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000
-4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL
-5,5.0,5,'five',false,NULL,NULL,NULL,NULL
-6,16,60,'',true,0,-1,-6,2010-12-31 23:59:59
-7,NULL,10,'NULL',false,127,-7,2,1400-01-01 00:00:00
-8,88,888,'eight',NULL,NULL,NULL,NULL,NULL
-9,-9,-99,'NULL',NULL,NULL,NULL,NULL,NULL
+1,1,1,'one',true,1,1,1,1987-05-19 00:00:00,0.000000001,1.00,1
+2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000,0.999999999,9999999999999999.99,99999999999999999999999999999999999999
+4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL,NULL,NULL,NULL
+5,5.0,5,'five',false,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+6,16,60,'',true,0,-1,-6,2010-12-31 23:59:59,-0.000000001,-1.00,-1
+7,NULL,10,'NULL',false,127,-7,2,1400-01-01 00:00:00,0.000000000,0.00,0
+8,88,888,'eight',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+9,-9,-99,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # SELECT, single row, all target cols
 insert into tdata
 select id, float_col, bigint_col, string_col, bool_col, tinyint_col, smallint_col,
-double_col, timestamp_col
+double_col, timestamp_col, NULL, NULL, NULL
 from functional.alltypes where id = 10
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-1,1,1,'one',true,1,1,1,1987-05-19 00:00:00
-2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
-3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000
-4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL
-5,5.0,5,'five',false,NULL,NULL,NULL,NULL
-6,16,60,'',true,0,-1,-6,2010-12-31 23:59:59
-7,NULL,10,'NULL',false,127,-7,2,1400-01-01 00:00:00
-8,88,888,'eight',NULL,NULL,NULL,NULL,NULL
-9,-9,-99,'NULL',NULL,NULL,NULL,NULL,NULL
-10,0,0,'0',true,0,0,0,2009-01-02 00:10:00.450000000
+1,1,1,'one',true,1,1,1,1987-05-19 00:00:00,0.000000001,1.00,1
+2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000,0.999999999,9999999999999999.99,99999999999999999999999999999999999999
+4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL,NULL,NULL,NULL
+5,5.0,5,'five',false,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+6,16,60,'',true,0,-1,-6,2010-12-31 23:59:59,-0.000000001,-1.00,-1
+7,NULL,10,'NULL',false,127,-7,2,1400-01-01 00:00:00,0.000000000,0.00,0
+8,88,888,'eight',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+9,-9,-99,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+10,0,0,'0',true,0,0,0,2009-01-02 00:10:00.450000000,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # SELECT, single row, subset of cols
@@ -151,21 +153,21 @@ from functional.alltypes where id = 11
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-1,1,1,'one',true,1,1,1,1987-05-19 00:00:00
-2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
-3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000
-4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL
-5,5.0,5,'five',false,NULL,NULL,NULL,NULL
-6,16,60,'',true,0,-1,-6,2010-12-31 23:59:59
-7,NULL,10,'NULL',false,127,-7,2,1400-01-01 00:00:00
-8,88,888,'eight',NULL,NULL,NULL,NULL,NULL
-9,-9,-99,'NULL',NULL,NULL,NULL,NULL,NULL
-10,0,0,'0',true,0,0,0,2009-01-02 00:10:00.450000000
-11,NULL,10,'NULL',false,NULL,NULL,10.1,2009-01-02 00:11:00.450000000
+1,1,1,'one',true,1,1,1,1987-05-19 00:00:00,0.000000001,1.00,1
+2,NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+3,NaN,9223372036854775807,'',true,-128,32767,-Infinity,9999-12-31 23:59:59.999999000,0.999999999,9999999999999999.99,99999999999999999999999999999999999999
+4,NULL,NULL,'NULL',true,NULL,NULL,0,NULL,NULL,NULL,NULL
+5,5.0,5,'five',false,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+6,16,60,'',true,0,-1,-6,2010-12-31 23:59:59,-0.000000001,-1.00,-1
+7,NULL,10,'NULL',false,127,-7,2,1400-01-01 00:00:00,0.000000000,0.00,0
+8,88,888,'eight',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+9,-9,-99,'NULL',NULL,NULL,NULL,NULL,NULL,NULL,NULL,NULL
+10,0,0,'0',true,0,0,0,2009-01-02 00:10:00.450000000,NULL,NULL,NULL
+11,NULL,10,'NULL',false,NULL,NULL,10.1,2009-01-02 00:11:00.450000000,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 delete tdata
@@ -175,18 +177,18 @@ delete tdata
 # SELECT, multiple rows, all target cols
 insert into tdata
 select id, float_col, bigint_col, string_col, bool_col, tinyint_col, smallint_col,
-double_col, timestamp_col
+double_col, timestamp_col, NULL, NULL, NULL
 from functional.alltypes where id < 2
 ---- RUNTIME_PROFILE
 NumModifiedRows: 2
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-0,0,0,'0',true,0,0,0,2009-01-01 00:00:00
-1,1.100000023841858,10,'1',false,1,1,10.1,2009-01-01 00:01:00
+0,0,0,'0',true,0,0,0,2009-01-01 00:00:00,NULL,NULL,NULL
+1,1.100000023841858,10,'1',false,1,1,10.1,2009-01-01 00:01:00,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # SELECT, multiple rows, subset of cols
@@ -197,20 +199,20 @@ from functional.alltypes where id > 2 and id < 6
 NumModifiedRows: 3
 NumRowErrors: 0
 ---- LABELS
-ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS
+ID, VALF, VALI, VALV, VALB, VALT, VALS, VALD, TS, DECIMAL4, DECIMAL8, DECIMAL16
 ---- DML_RESULTS: tdata
-0,0,0,'0',true,0,0,0,2009-01-01 00:00:00
-1,1.100000023841858,10,'1',false,1,1,10.1,2009-01-01 00:01:00
-3,NULL,NULL,'NULL',NULL,3,3,30.3,2009-01-01 00:03:00.300000000
-4,NULL,NULL,'NULL',NULL,4,4,40.4,2009-01-01 00:04:00.600000000
-5,NULL,NULL,'NULL',NULL,5,5,50.5,2009-01-01 00:05:00.100000000
+0,0,0,'0',true,0,0,0,2009-01-01 00:00:00,NULL,NULL,NULL
+1,1.100000023841858,10,'1',false,1,1,10.1,2009-01-01 00:01:00,NULL,NULL,NULL
+3,NULL,NULL,'NULL',NULL,3,3,30.3,2009-01-01 00:03:00.300000000,NULL,NULL,NULL
+4,NULL,NULL,'NULL',NULL,4,4,40.4,2009-01-01 00:04:00.600000000,NULL,NULL,NULL
+5,NULL,NULL,'NULL',NULL,5,5,50.5,2009-01-01 00:05:00.100000000,NULL,NULL,NULL
 ---- TYPES
-INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP
+INT,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,TIMESTAMP,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # Make sure we can insert empty strings into string columns and that we can scan them
 # back.
-insert into tdata values (320, 2.0, 932, cast('' as string), false, 0, 0, 0, NULL)
+insert into tdata values (320, 2.0, 932, cast('' as string), false, 0, 0, 0, NULL, NULL, NULL, NULL)
 ---- RESULTS
 : 1
 ---- RUNTIME_PROFILE
@@ -226,7 +228,7 @@ INT,STRING,BOOLEAN
 ====
 ---- QUERY
 insert into tdata values
-(666, cast(1.2 as float), 43, cast('z' as string), true, 0, 0, 0, NULL)
+(666, cast(1.2 as float), 43, cast('z' as string), true, 0, 0, 0, NULL, NULL, NULL, NULL)
 ---- RESULTS
 : 1
 ---- RUNTIME_PROFILE
@@ -236,7 +238,7 @@ NumRowErrors: 0
 ---- QUERY
 # insert row with primary key that already exists
 insert into tdata values
-(666, cast(1.2 as float), 43, cast('z' as VARCHAR(20)), true, 0, 0, 0, NULL)
+(666, cast(1.2 as float), 43, cast('z' as VARCHAR(20)), true, 0, 0, 0, NULL, NULL, NULL, NULL)
 ---- RESULTS
 : 0
 ---- RUNTIME_PROFILE
@@ -319,7 +321,8 @@ NumRowErrors: 0
 # Table with default values
 create table tbl_with_defaults (a int primary key, b int null default 10,
   c int not null default 100, d int default 1000, e int null, f int not null,
-  g string default 'test', h boolean default true) partition by hash (a)
+  g string default 'test', h boolean default true,
+  i decimal(9, 2) default 1111.11) partition by hash (a)
   partitions 3 stored as kudu
 ---- RESULTS
 ====
@@ -329,62 +332,62 @@ insert into tbl_with_defaults (a, f) values (1, 1), (2, 2), (3, 3), (4, 4)
 NumModifiedRows: 4
 NumRowErrors: 0
 ---- LABELS
-A, B, C, D, E, F, G, H
+A, B, C, D, E, F, G, H, I
 ---- DML_RESULTS: tbl_with_defaults
-1,10,100,1000,NULL,1,'test',true
-2,10,100,1000,NULL,2,'test',true
-3,10,100,1000,NULL,3,'test',true
-4,10,100,1000,NULL,4,'test',true
+1,10,100,1000,NULL,1,'test',true,1111.11
+2,10,100,1000,NULL,2,'test',true,1111.11
+3,10,100,1000,NULL,3,'test',true,1111.11
+4,10,100,1000,NULL,4,'test',true,1111.11
 ---- TYPES
-INT,INT,INT,INT,INT,INT,STRING,BOOLEAN
+INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL
 ====
 ---- QUERY
-insert into tbl_with_defaults values (5, 5, 5, 5, 5, 5, 'row', false)
+insert into tbl_with_defaults values (5, 5, 5, 5, 5, 5, 'row', false, 55555.55)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-A, B, C, D, E, F, G, H
+A, B, C, D, E, F, G, H, I
 ---- DML_RESULTS: tbl_with_defaults
-1,10,100,1000,NULL,1,'test',true
-2,10,100,1000,NULL,2,'test',true
-3,10,100,1000,NULL,3,'test',true
-4,10,100,1000,NULL,4,'test',true
-5,5,5,5,5,5,'row',false
+1,10,100,1000,NULL,1,'test',true,1111.11
+2,10,100,1000,NULL,2,'test',true,1111.11
+3,10,100,1000,NULL,3,'test',true,1111.11
+4,10,100,1000,NULL,4,'test',true,1111.11
+5,5,5,5,5,5,'row',false,55555.55
 ---- TYPES
-INT,INT,INT,INT,INT,INT,STRING,BOOLEAN
+INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL
 ====
 ---- QUERY
-alter table tbl_with_defaults add columns (i int null, j int not null default 10000)
+alter table tbl_with_defaults add columns (j int null, k int not null default 10000)
 ---- RESULTS
 ====
 ---- QUERY
 select * from tbl_with_defaults
 ---- RESULTS
-1,10,100,1000,NULL,1,'test',true,NULL,10000
-2,10,100,1000,NULL,2,'test',true,NULL,10000
-3,10,100,1000,NULL,3,'test',true,NULL,10000
-4,10,100,1000,NULL,4,'test',true,NULL,10000
-5,5,5,5,5,5,'row',false,NULL,10000
+1,10,100,1000,NULL,1,'test',true,1111.11,NULL,10000
+2,10,100,1000,NULL,2,'test',true,1111.11,NULL,10000
+3,10,100,1000,NULL,3,'test',true,1111.11,NULL,10000
+4,10,100,1000,NULL,4,'test',true,1111.11,NULL,10000
+5,5,5,5,5,5,'row',false,55555.55,NULL,10000
 ---- TYPES
-INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,INT,INT
+INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL,INT,INT
 ====
 ---- QUERY
-insert into tbl_with_defaults values (6,6,6,6,6,6,'another row',false,6,6)
+insert into tbl_with_defaults values (6,6,6,6,6,6,'another row',false,66666.66,6,6)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-A, B, C, D, E, F, G, H, I, J
+A, B, C, D, E, F, G, H, I, J, K
 ---- DML_RESULTS: tbl_with_defaults
-1,10,100,1000,NULL,1,'test',true,NULL,10000
-2,10,100,1000,NULL,2,'test',true,NULL,10000
-3,10,100,1000,NULL,3,'test',true,NULL,10000
-4,10,100,1000,NULL,4,'test',true,NULL,10000
-5,5,5,5,5,5,'row',false,NULL,10000
-6,6,6,6,6,6,'another row',false,6,6
+1,10,100,1000,NULL,1,'test',true,1111.11,NULL,10000
+2,10,100,1000,NULL,2,'test',true,1111.11,NULL,10000
+3,10,100,1000,NULL,3,'test',true,1111.11,NULL,10000
+4,10,100,1000,NULL,4,'test',true,1111.11,NULL,10000
+5,5,5,5,5,5,'row',false,55555.55,NULL,10000
+6,6,6,6,6,6,'another row',false,66666.66,6,6
 ---- TYPES
-INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,INT,INT
+INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL,INT,INT
 ====
 ---- QUERY
 # IMPALA-5217: Try to insert NULL to a 'NOT NULL' col with a target col list that leaves
@@ -402,17 +405,17 @@ insert into tbl_with_defaults (a, b, d, f) values (0, 0, null, 0)
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-A, B, C, D, E, F, G, H, I, J
+A, B, C, D, E, F, G, H, I, J, K
 ---- DML_RESULTS: tbl_with_defaults
-0,0,100,NULL,NULL,0,'test',true,NULL,10000
-1,10,100,1000,NULL,1,'test',true,NULL,10000
-2,10,100,1000,NULL,2,'test',true,NULL,10000
-3,10,100,1000,NULL,3,'test',true,NULL,10000
-4,10,100,1000,NULL,4,'test',true,NULL,10000
-5,5,5,5,5,5,'row',false,NULL,10000
-6,6,6,6,6,6,'another row',false,6,6
+0,0,100,NULL,NULL,0,'test',true,1111.11,NULL,10000
+1,10,100,1000,NULL,1,'test',true,1111.11,NULL,10000
+2,10,100,1000,NULL,2,'test',true,1111.11,NULL,10000
+3,10,100,1000,NULL,3,'test',true,1111.11,NULL,10000
+4,10,100,1000,NULL,4,'test',true,1111.11,NULL,10000
+5,5,5,5,5,5,'row',false,55555.55,NULL,10000
+6,6,6,6,6,6,'another row',false,66666.66,6,6
 ---- TYPES
-INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,INT,INT
+INT,INT,INT,INT,INT,INT,STRING,BOOLEAN,DECIMAL,INT,INT
 ====
 ---- QUERY
 create table multiple_partition_cols (x bigint, y bigint, z string, primary key(x, y))


[11/15] impala git commit: IMPALA-5752: Add support for DECIMAL on Kudu tables

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
index 8d0d149..8520677 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_update.test
@@ -3,18 +3,19 @@
 create table tdata
   (id int primary key, name string null, valf float null, vali bigint null,
    valv string null, valb boolean null, valt tinyint null, vals smallint null,
-   vald double null)
+   vald double null, valdec4 decimal(9,9) null, valdec8 decimal(18,2) null,
+   valdec16 decimal(38, 0) null)
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES <= 10000) STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY
 insert into tdata values
-(1, "martin", 1.0, 232232323, cast('a' as string), true, 1, 2, 3),
-(2, "david", cast(1.0 as float), 99398493939, cast('b' as string), false, 4, 5, 6),
-(3, "todd", cast(1.0 as float), 993393939, "c", true, 7, 8, 9),
-(40, "he", cast(0.0 as float), 43, cast('e' as string), false, 50, 60, 70),
-(120, "she", cast(0.0 as float), 99, cast('f' as string), true, -1, 0, 1)
+(1, "martin", 1.0, 232232323, cast('a' as string), true, 1, 2, 3, 0.000000001, 2.22, 3),
+(2, "david", cast(1.0 as float), 99398493939, cast('b' as string), false, 4, 5, 6, 0.000000004, 5.55, 6),
+(3, "todd", cast(1.0 as float), 993393939, "c", true, 7, 8, 9, 0.000000007, 8.88, 9),
+(40, "he", cast(0.0 as float), 43, cast('e' as string), false, 50, 60, 70, 0.000000050, 66.60, 70),
+(120, "she", cast(0.0 as float), 99, cast('f' as string), true, -1, 0, 1, -0.000000001, 0.00, 1)
 ---- RESULTS
 : 5
 ---- RUNTIME_PROFILE
@@ -30,15 +31,15 @@ update tdata set vali=43 where id = 1
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'martin',1.0,43,'a',true,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',1.0,993393939,'c',true,7,8,9
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,99,'f',true,-1,0,1
+1,'martin',1.0,43,'a',true,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,99,'f',true,-1,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, equality on key, string
@@ -49,15 +50,15 @@ update tdata set valv=cast('aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' as varchar(20)) wh
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',true,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',1.0,993393939,'c',true,7,8,9
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,99,'f',true,-1,0,1
+1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',true,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,99,'f',true,-1,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, equality on key, boolean
@@ -66,15 +67,15 @@ update tdata set valb=false where id = 1
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',1.0,993393939,'c',true,7,8,9
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,99,'f',true,-1,0,1
+1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,99,'f',true,-1,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, equality on key, set to NULL
@@ -83,15 +84,15 @@ update tdata set name=null where id = 40
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',1.0,993393939,'c',true,7,8,9
-40,'NULL',0.0,43,'e',false,50,60,70
-120,'she',0.0,99,'f',true,-1,0,1
+1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'NULL',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,99,'f',true,-1,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, equlity on key, set from NULL
@@ -100,15 +101,15 @@ update tdata set name='he' where id = 40
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',1.0,993393939,'c',true,7,8,9
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,99,'f',true,-1,0,1
+1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,99,'f',true,-1,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, equality on key, boundary value
@@ -117,15 +118,15 @@ update tdata set vali = max_bigint() where id = 120
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',1.0,993393939,'c',true,7,8,9
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',true,-1,0,1
+1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',true,-1,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, inequality on key, float
@@ -134,15 +135,15 @@ update tdata set valf = -1 where id > 2 and id < 4
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',-1.0,993393939,'c',true,7,8,9
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',true,-1,0,1
+1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',true,-1,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, inequality on key, tinyint
@@ -151,15 +152,15 @@ update tdata set valt = 10 where id > 100
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',-1.0,993393939,'c',true,7,8,9
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',true,10,0,1
+1,'martin',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',true,10,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, equality on non-key, string
@@ -168,15 +169,15 @@ update tdata set name='unknown' where name = 'martin'
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',-1.0,993393939,'c',true,7,8,9
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',true,10,0,1
+1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,993393939,'c',true,7,8,9,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',true,10,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # single row, inequality on non-key, double
@@ -185,15 +186,32 @@ update tdata set vald = 0 where valf < 0
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,99398493939,'b',false,4,5,6
-3,'todd',-1.0,993393939,'c',true,7,8,0
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',true,10,0,1
+1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,993393939,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',true,10,0,1,-0.000000001,0.00,1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
+====
+---- QUERY
+# single row, inequality on non-key, decimal
+update tdata set valdec4 = 0.000000001, valdec16 = -1 where valdec4 < 0
+---- RUNTIME_PROFILE
+NumModifiedRows: 1
+NumRowErrors: 0
+---- LABELS
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
+---- DML_RESULTS: tdata
+1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,99398493939,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,993393939,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',true,10,0,1,0.000000001,0.00,-1
+---- TYPES
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # multiple rows, predicate on key
@@ -202,15 +220,15 @@ update tdata set vali=43 where id > 1 and id < 10
 NumModifiedRows: 2
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,43,'b',false,4,5,6
-3,'todd',-1.0,43,'c',true,7,8,0
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',true,10,0,1
+1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,43,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,43,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',true,10,0,1,0.000000001,0.00,-1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # multiple rows, predicate on non-key
@@ -219,15 +237,15 @@ update tdata set valb=false where name LIKE '%he'
 NumModifiedRows: 2
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,43,'b',false,4,5,6
-3,'todd',-1.0,43,'c',true,7,8,0
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',false,10,0,1
+1,'unknown',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,43,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,43,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',false,10,0,1,0.000000001,0.00,-1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # update the same row multiple times: cross join produces 5 identical updates
@@ -236,15 +254,15 @@ update a set a.name='they' from tdata a, tdata b where a.id = 1
 NumModifiedRows: 5
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'they',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,43,'b',false,4,5,6
-3,'todd',-1.0,43,'c',true,7,8,0
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',false,10,0,1
+1,'they',1.0,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,43,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,43,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',false,10,0,1,0.000000001,0.00,-1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # value of update is a constant expression
@@ -253,15 +271,15 @@ update tdata set valf = 1 + 2 where id = 1
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'they',3,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,43,'b',false,4,5,6
-3,'todd',-1.0,43,'c',true,7,8,0
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',false,10,0,1
+1,'they',3,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,43,'b',false,4,5,6,0.000000004,5.55,6
+3,'todd',-1.0,43,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',false,10,0,1,0.000000001,0.00,-1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # value of update is a non-constant expression
@@ -270,15 +288,15 @@ update tdata set name = concat(name, name) where id % 2 = 1
 NumModifiedRows: 2
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'theythey',3,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,43,'b',false,4,5,6
-3,'toddtodd',-1.0,43,'c',true,7,8,0
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',false,10,0,1
+1,'theythey',3,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,43,'b',false,4,5,6,0.000000004,5.55,6
+3,'toddtodd',-1.0,43,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',false,10,0,1,0.000000001,0.00,-1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # error - update key column
@@ -293,15 +311,15 @@ update tdata set name = 'none' where id = 10
 NumModifiedRows: 0
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'theythey',3,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,43,'b',false,4,5,6
-3,'toddtodd',-1.0,43,'c',true,7,8,0
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',false,10,0,1
+1,'theythey',3,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,43,'b',false,4,5,6,0.000000004,5.55,6
+3,'toddtodd',-1.0,43,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',false,10,0,1,0.000000001,0.00,-1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 # Try to update a row with a primary key value that is not covered by the existing range
@@ -311,20 +329,20 @@ update tdata set vali = 10 where id = 10001
 NumModifiedRows: 0
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC4, VALDEC8, VALDEC16
 ---- DML_RESULTS: tdata
-1,'theythey',3,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3
-2,'david',1.0,43,'b',false,4,5,6
-3,'toddtodd',-1.0,43,'c',true,7,8,0
-40,'he',0.0,43,'e',false,50,60,70
-120,'she',0.0,9223372036854775807,'f',false,10,0,1
+1,'theythey',3,43,'aaaaaaaaaaaaaaaaaaaa',false,1,2,3,0.000000001,2.22,3
+2,'david',1.0,43,'b',false,4,5,6,0.000000004,5.55,6
+3,'toddtodd',-1.0,43,'c',true,7,8,0,0.000000007,8.88,9
+40,'he',0.0,43,'e',false,50,60,70,0.000000050,66.60,70
+120,'she',0.0,9223372036854775807,'f',false,10,0,1,0.000000001,0.00,-1
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL,DECIMAL,DECIMAL
 ====
 ---- QUERY
 insert into tdata
 select id, string_col, float_col, bigint_col, string_col, bool_col, tinyint_col,
-smallint_col, double_col from functional_kudu.alltypes
+smallint_col, double_col, NULL, NULL, NULL from functional_kudu.alltypes
 ---- RESULTS
 : 7295
 ---- RUNTIME_PROFILE

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test b/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
index 57aaa46..1c12f33 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/kudu_upsert.test
@@ -3,149 +3,149 @@
 create table tdata
   (id int primary key, name string null, valf float null, vali bigint null,
    valv string null, valb boolean null, valt tinyint null, vals smallint null,
-   vald double null)
+   vald double null, valdec decimal(9, 0) null)
   PARTITION BY RANGE (PARTITION VALUES < 10, PARTITION 10 <= VALUES < 30,
   PARTITION 30 <= VALUES) STORED AS KUDU
 ---- RESULTS
 ====
 ---- QUERY
 insert into table tdata values
-(40,'he',0,43,'e',false,35,36,1.2),
-(1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0),
-(2,'david',1,43,'b',false,0,0,0),
-(3,'todd',1,43,'c',true,3,3,3)
+(40,'he',0,43,'e',false,35,36,1.2,37),
+(1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3),
+(2,'david',1,43,'b',false,0,0,0,0),
+(3,'todd',1,43,'c',true,3,3,3,3)
 ---- RESULTS
 : 4
 ====
 ---- QUERY
 # VALUES, single row, all cols, results in insert
-upsert into table tdata values (4, 'a', 0, 1, 'b', false, 1, 2, 1.5)
+upsert into table tdata values (4, 'a', 0, 1, 'b', false, 1, 2, 1.5, 4)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'a',0,1,'b',false,1,2,1.5
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'a',0,1,'b',false,1,2,1.5,4
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all cols, results in update
-upsert into table tdata values (4, 'b', -1, 1, 'a', true, 2, 3, 2.5)
+upsert into table tdata values (4, 'b', -1, 1, 'a', true, 2, 3, 2.5, 5)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'b',-1,1,'a',true,2,3,2.5
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'b',-1,1,'a',true,2,3,2.5,5
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all cols, insert NULL all types
-upsert into table tdata values (10, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)
+upsert into table tdata values (10, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'b',-1,1,'a',true,2,3,2.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'b',-1,1,'a',true,2,3,2.5,5
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all cols, update NULL all types
-upsert into table tdata values (4, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)
+upsert into table tdata values (4, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all cols, update from NULL all types
-upsert into table tdata values (4, 'four', 5, 6, 'f', true, 7, 8, 7.5)
+upsert into table tdata values (4, 'four', 5, 6, 'f', true, 7, 8, 7.5, 9)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'four',5,6,'f',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'four',5,6,'f',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all cols, insert boundary values
 upsert into table tdata values
 (max_int(), '', cast('nan' as float), min_bigint(), '', true, max_tinyint(),
-  min_smallint(), cast('inf' as double))
+  min_smallint(), cast('inf' as double), cast(999999999 as decimal(9, 0)))
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'four',5,6,'f',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',NaN,-9223372036854775808,'',true,127,-32768,Infinity
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'four',5,6,'f',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',NaN,-9223372036854775808,'',true,127,-32768,Infinity,999999999
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all cols, update boundary values
 upsert into table tdata values
 (max_int(), '', cast('-inf' as float), max_bigint(), '', true, min_tinyint(),
-  max_smallint(), cast('nan' as double))
+  max_smallint(), cast('nan' as double), cast(-999999999 as decimal(9, 0)))
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'four',5,6,'f',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'four',5,6,'f',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, subset of cols, results in insert
@@ -154,18 +154,18 @@ upsert into table tdata (id, name, vali, valb, vald) values (5, 'five', -5, NULL
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'four',5,6,'f',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
-5,'five',NULL,-5,'NULL',NULL,NULL,NULL,0.5
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'four',5,6,'f',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'five',NULL,-5,'NULL',NULL,NULL,NULL,0.5,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, subset of cols, results in update
@@ -174,44 +174,44 @@ upsert into table tdata (id, name, valf, valv, valb) values (5, NULL, 0, 'six',
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0
-2,'david',1,43,'b',false,0,0,0
-3,'todd',1,43,'c',true,3,3,3
-4,'four',5,6,'f',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
-5,'NULL',0,-5,'six',false,NULL,NULL,0.5
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'unknown',1,43,'aaaaaaaaaaaaaaaaaaaa',false,-1,-2,0,-3
+2,'david',1,43,'b',false,0,0,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'four',5,6,'f',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'NULL',0,-5,'six',false,NULL,NULL,0.5,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, multiple rows, all cols, no errors
 upsert into table tdata values
-(1, 'one', NULL, 44, 'a', true, -1, -2, 0),
-(6, '', -6, 40, 'b', NULL, 0, 0, 10),
-(7, 'seven', 0, min_bigint(), NULL, true, 7, 1, 2),
-(2, 'you', cast('inf' as float), 0, 't', false, NULL, min_smallint(), 0)
+(1, 'one', NULL, 44, 'a', true, -1, -2, 0, 0),
+(6, '', -6, 40, 'b', NULL, 0, 0, 10, 11),
+(7, 'seven', 0, min_bigint(), NULL, true, 7, 1, 2, 3),
+(2, 'you', cast('inf' as float), 0, 't', false, NULL, min_smallint(), 0, 0)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 4
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'one',NULL,44,'a',true,-1,-2,0
-2,'you',Infinity,0,'t',false,NULL,-32768,0
-3,'todd',1,43,'c',true,3,3,3
-4,'four',5,6,'f',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
-5,'NULL',0,-5,'six',false,NULL,NULL,0.5
-6,'',-6,40,'b',NULL,0,0,10
-7,'seven',0,-9223372036854775808,'NULL',true,7,1,2
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'one',NULL,44,'a',true,-1,-2,0,0
+2,'you',Infinity,0,'t',false,NULL,-32768,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'four',5,6,'f',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'NULL',0,-5,'six',false,NULL,NULL,0.5,NULL
+6,'',-6,40,'b',NULL,0,0,10,11
+7,'seven',0,-9223372036854775808,'NULL',true,7,1,2,3
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, multiple rows, subset of cols, no errors
@@ -224,72 +224,72 @@ upsert into table tdata (id, valb, name, vali) values
 NumModifiedRows: 4
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'NULL',NULL,1,'a',true,-1,-2,0
-2,'you',Infinity,0,'t',false,NULL,-32768,0
-3,'todd',1,43,'c',true,3,3,3
-4,'four',5,6,'f',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
-5,'five',0,10,'six',NULL,NULL,NULL,0.5
-6,'',-6,40,'b',NULL,0,0,10
-7,'seven',0,-9223372036854775808,'NULL',true,7,1,2
-8,'hello',NULL,2,'NULL',false,NULL,NULL,NULL
-9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'NULL',NULL,1,'a',true,-1,-2,0,0
+2,'you',Infinity,0,'t',false,NULL,-32768,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'four',5,6,'f',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'five',0,10,'six',NULL,NULL,NULL,0.5,NULL
+6,'',-6,40,'b',NULL,0,0,10,11
+7,'seven',0,-9223372036854775808,'NULL',true,7,1,2,3
+8,'hello',NULL,2,'NULL',false,NULL,NULL,NULL,NULL
+9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # SELECT, all cols, single row, no errors
 upsert into table tdata
-select id, 'a', valf, vali, valv, NULL, valt, vals, 3 from tdata where id = 1
+select id, 'a', valf, vali, valv, NULL, valt, vals, 3, valdec from tdata where id = 1
 ---- RUNTIME_PROFILE
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'he',0,43,'e',false,35,36,1.2
-1,'a',NULL,1,'a',NULL,-1,-2,3
-2,'you',Infinity,0,'t',false,NULL,-32768,0
-3,'todd',1,43,'c',true,3,3,3
-4,'four',5,6,'f',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
-5,'five',0,10,'six',NULL,NULL,NULL,0.5
-6,'',-6,40,'b',NULL,0,0,10
-7,'seven',0,-9223372036854775808,'NULL',true,7,1,2
-8,'hello',NULL,2,'NULL',false,NULL,NULL,NULL
-9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL
+40,'he',0,43,'e',false,35,36,1.2,37
+1,'a',NULL,1,'a',NULL,-1,-2,3,0
+2,'you',Infinity,0,'t',false,NULL,-32768,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'four',5,6,'f',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'five',0,10,'six',NULL,NULL,NULL,0.5,NULL
+6,'',-6,40,'b',NULL,0,0,10,11
+7,'seven',0,-9223372036854775808,'NULL',true,7,1,2,3
+8,'hello',NULL,2,'NULL',false,NULL,NULL,NULL,NULL
+9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # SELECT, all cols, multiple rows, no errors
 upsert into table tdata
-select id, valv, valf, vali, name, valb, valt, vals, vald from tdata where id % 2 = 0
+select id, valv, valf, vali, name, valb, valt, vals, vald, valdec from tdata where id % 2 = 0
 ---- RUNTIME_PROFILE
 NumModifiedRows: 6
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'e',0,43,'he',false,35,36,1.2
-1,'a',NULL,1,'a',NULL,-1,-2,3
-2,'t',Infinity,0,'you',false,NULL,-32768,0
-3,'todd',1,43,'c',true,3,3,3
-4,'f',5,6,'four',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
-5,'five',0,10,'six',NULL,NULL,NULL,0.5
-6,'b',-6,40,'',NULL,0,0,10
-7,'seven',0,-9223372036854775808,'NULL',true,7,1,2
-8,'NULL',NULL,2,'hello',false,NULL,NULL,NULL
-9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL
+40,'e',0,43,'he',false,35,36,1.2,37
+1,'a',NULL,1,'a',NULL,-1,-2,3,0
+2,'t',Infinity,0,'you',false,NULL,-32768,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'f',5,6,'four',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'five',0,10,'six',NULL,NULL,NULL,0.5,NULL
+6,'b',-6,40,'',NULL,0,0,10,11
+7,'seven',0,-9223372036854775808,'NULL',true,7,1,2,3
+8,'NULL',NULL,2,'hello',false,NULL,NULL,NULL,NULL
+9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # SELECT, subset of cols, single row, no errors
@@ -299,23 +299,23 @@ select int_col, string_col, bigint_col from functional.alltypes where id = 0
 NumModifiedRows: 1
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'e',0,43,'he',false,35,36,1.2
-1,'a',NULL,1,'a',NULL,-1,-2,3
-2,'t',Infinity,0,'you',false,NULL,-32768,0
-3,'todd',1,43,'c',true,3,3,3
-4,'f',5,6,'four',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
-5,'five',0,10,'six',NULL,NULL,NULL,0.5
-6,'b',-6,40,'',NULL,0,0,10
-7,'seven',0,-9223372036854775808,'NULL',true,7,1,2
-8,'NULL',NULL,2,'hello',false,NULL,NULL,NULL
-9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL
-0,'NULL',NULL,0,'0',NULL,NULL,NULL,NULL
+40,'e',0,43,'he',false,35,36,1.2,37
+1,'a',NULL,1,'a',NULL,-1,-2,3,0
+2,'t',Infinity,0,'you',false,NULL,-32768,0,0
+3,'todd',1,43,'c',true,3,3,3,3
+4,'f',5,6,'four',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'five',0,10,'six',NULL,NULL,NULL,0.5,NULL
+6,'b',-6,40,'',NULL,0,0,10,11
+7,'seven',0,-9223372036854775808,'NULL',true,7,1,2,3
+8,'NULL',NULL,2,'hello',false,NULL,NULL,NULL,NULL
+9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL,NULL
+0,'NULL',NULL,0,'0',NULL,NULL,NULL,NULL,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # SELECT, subset of cols, multiple rows, no errors
@@ -325,27 +325,53 @@ select int_col, bool_col, string_col, tinyint_col from functional.alltypes where
 NumModifiedRows: 4
 NumRowErrors: 0
 ---- LABELS
-ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
 ---- DML_RESULTS: tdata
-40,'e',0,43,'he',false,35,36,1.2
-1,'1',NULL,1,'a',false,1,-2,3
-2,'2',Infinity,0,'you',true,2,-32768,0
-3,'3',1,43,'c',false,3,3,3
-4,'f',5,6,'four',true,7,8,7.5
-10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL
-2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN
-5,'five',0,10,'six',NULL,NULL,NULL,0.5
-6,'b',-6,40,'',NULL,0,0,10
-7,'seven',0,-9223372036854775808,'NULL',true,7,1,2
-8,'NULL',NULL,2,'hello',false,NULL,NULL,NULL
-9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL
-0,'0',NULL,0,'0',true,0,NULL,NULL
+40,'e',0,43,'he',false,35,36,1.2,37
+1,'1',NULL,1,'a',false,1,-2,3,0
+2,'2',Infinity,0,'you',true,2,-32768,0,0
+3,'3',1,43,'c',false,3,3,3,3
+4,'f',5,6,'four',true,7,8,7.5,9
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'five',0,10,'six',NULL,NULL,NULL,0.5,NULL
+6,'b',-6,40,'',NULL,0,0,10,11
+7,'seven',0,-9223372036854775808,'NULL',true,7,1,2,3
+8,'NULL',NULL,2,'hello',false,NULL,NULL,NULL,NULL
+9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL,NULL
+0,'0',NULL,0,'0',true,0,NULL,NULL,NULL
 ---- TYPES
-INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
+====
+---- QUERY
+# SELECT, decimal column, multiple rows, no errors
+upsert into table tdata (id, valdec)
+select id, cast((valdec * 2) as decimal(9,0)) from tdata where valdec > 0
+---- RUNTIME_PROFILE
+NumModifiedRows: 5
+NumRowErrors: 0
+---- LABELS
+ID, NAME, VALF, VALI, VALV, VALB, VALT, VALS, VALD, VALDEC
+---- DML_RESULTS: tdata
+40,'e',0,43,'he',false,35,36,1.2,74
+1,'1',NULL,1,'a',false,1,-2,3,0
+2,'2',Infinity,0,'you',true,2,-32768,0,0
+3,'3',1,43,'c',false,3,3,3,6
+4,'f',5,6,'four',true,7,8,7.5,18
+10,'NULL',NULL,NULL,'NULL',NULL,NULL,NULL,NULL,NULL
+2147483647,'',-Infinity,9223372036854775807,'',true,-128,32767,NaN,-999999999
+5,'five',0,10,'six',NULL,NULL,NULL,0.5,NULL
+6,'b',-6,40,'',NULL,0,0,10,22
+7,'seven',0,-9223372036854775808,'NULL',true,7,1,2,6
+8,'NULL',NULL,2,'hello',false,NULL,NULL,NULL,NULL
+9,'nine',NULL,9,'NULL',true,NULL,NULL,NULL,NULL
+0,'0',NULL,0,'0',true,0,NULL,NULL,NULL
+---- TYPES
+INT,STRING,FLOAT,BIGINT,STRING,BOOLEAN,TINYINT,SMALLINT,DOUBLE,DECIMAL
 ====
 ---- QUERY
 # VALUES, single row, all cols, null for non-nullable column
-upsert into table tdata values (null, '', 0, 0, cast('' as VARCHAR(20)), false, 0, 0, 0)
+upsert into table tdata values (null, '', 0, 0, cast('' as VARCHAR(20)), false, 0, 0, 0, null)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 0
 NumRowErrors: 1
@@ -360,10 +386,10 @@ NumRowErrors: 1
 ---- QUERY
 # VALUES, multiple rows, all cols, null for non-nullable column
 upsert into table tdata values
-(3,'todd',1,43,'c',true,3,3,3),
-(4,'four',5,6,'f',true,7,8,7.5),
-(6,'',-6,40,'b',NULL,0,0,10),
-(NULL,'seven',0,0,'NULL',true,7,1,2)
+(3,'todd',1,43,'c',true,3,3,3,null),
+(4,'four',5,6,'f',true,7,8,7.5,null),
+(6,'',-6,40,'b',NULL,0,0,10,null),
+(NULL,'seven',0,0,'NULL',true,7,1,2,null)
 ---- RUNTIME_PROFILE
 NumModifiedRows: 3
 NumRowErrors: 1

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/tests/query_test/test_decimal_queries.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_decimal_queries.py b/tests/query_test/test_decimal_queries.py
index a250ae2..3a14ed3 100644
--- a/tests/query_test/test_decimal_queries.py
+++ b/tests/query_test/test_decimal_queries.py
@@ -43,7 +43,7 @@ class TestDecimalQueries(ImpalaTestSuite):
     cls.ImpalaTestMatrix.add_constraint(lambda v:\
         (v.get_value('table_format').file_format == 'text' and
          v.get_value('table_format').compression_codec == 'none') or
-         v.get_value('table_format').file_format == 'parquet')
+         v.get_value('table_format').file_format in ['parquet', 'kudu'])
 
   def test_queries(self, vector):
     self.run_test_case('QueryTest/decimal', vector)
@@ -60,7 +60,7 @@ class TestDecimalExprs(ImpalaTestSuite):
   def add_test_dimensions(cls):
     super(TestDecimalExprs, cls).add_test_dimensions()
     cls.ImpalaTestMatrix.add_constraint(lambda v:
-        (v.get_value('table_format').file_format == 'parquet'))
+        (v.get_value('table_format').file_format in ['parquet', 'kudu']))
 
   def test_exprs(self, vector):
     self.run_test_case('QueryTest/decimal-exprs', vector)

http://git-wip-us.apache.org/repos/asf/impala/blob/84fffd46/tests/query_test/test_kudu.py
----------------------------------------------------------------------
diff --git a/tests/query_test/test_kudu.py b/tests/query_test/test_kudu.py
index ddf59f2..3d7b727 100644
--- a/tests/query_test/test_kudu.py
+++ b/tests/query_test/test_kudu.py
@@ -357,7 +357,7 @@ class TestKuduOperations(KuduTestSuite):
       pytest.skip("Only runs in exhaustive to reduce core time.")
     table_name = "%s.storage_attrs" % unique_database
     types = ['boolean', 'tinyint', 'smallint', 'int', 'bigint', 'float', 'double', \
-        'string', 'timestamp']
+        'string', 'timestamp', 'decimal(9, 2)', 'decimal(18)', 'decimal(38, 38)']
 
     create_query = "create table %s (id int primary key" % table_name
     for t in types:
@@ -485,6 +485,7 @@ class TestCreateExternalTable(KuduTestSuite):
 
   def test_col_types(self, cursor, kudu_client):
     """Check that a table can be created using all available column types."""
+    # TODO: Add DECIMAL when the Kudu python client supports decimal
     kudu_types = [STRING, BOOL, DOUBLE, FLOAT, INT16, INT32, INT64, INT8]
     with self.temp_kudu_table(kudu_client, kudu_types) as kudu_table:
       impala_table_name = self.get_kudu_table_base_name(kudu_table.name)


[10/15] impala git commit: IMPALA-6219: actually turn on AES-GCM

Posted by ta...@apache.org.
IMPALA-6219: actually turn on AES-GCM

There was a typo that prevented it from matching the output of cpuinfo
correctly.

Testing:
Manually verified that impalad.INFO contains the expected hardware
flags:

  Hardware Supports:
    ssse3
    sse4_1
    sse4_2
    popcnt
    avx
    avx2
    pclmulqdq

Added temporary logging to verify that openssl-util-test and impala with
--disk_spill_encryption=true were going down the expected code path.

Added permanent logging at startup to report the default mode.

Change-Id: I64f23c493c6cc8d7e51ff45bbb305a71085e84c8
Reviewed-on: http://gerrit.cloudera.org:8080/9386
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins
Reviewed-on: http://gerrit.cloudera.org:8080/9421
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Tim Armstrong <ta...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/31d197ad
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/31d197ad
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/31d197ad

Branch: refs/heads/2.x
Commit: 31d197ad469a51213d9c5d650927f0c9b4010de2
Parents: e0c0918
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Wed Feb 21 14:02:15 2018 -0800
Committer: Tim Armstrong <ta...@cloudera.com>
Committed: Fri Feb 23 22:51:01 2018 +0000

----------------------------------------------------------------------
 be/src/common/init.cc       |  2 ++
 be/src/util/cpu-info.cc     |  2 +-
 be/src/util/openssl-util.cc |  6 +++---
 be/src/util/openssl-util.h  | 20 ++++++++++----------
 4 files changed, 16 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/31d197ad/be/src/common/init.cc
----------------------------------------------------------------------
diff --git a/be/src/common/init.cc b/be/src/common/init.cc
index 41d4549..745112a 100644
--- a/be/src/common/init.cc
+++ b/be/src/common/init.cc
@@ -239,6 +239,8 @@ void impala::InitCommonRuntime(int argc, char** argv, bool init_jvm,
   LOG(INFO) << MemInfo::DebugString();
   LOG(INFO) << OsInfo::DebugString();
   LOG(INFO) << "Process ID: " << getpid();
+  LOG(INFO) << "Default AES cipher mode for spill-to-disk: "
+            << EncryptionKey::ModeToString(EncryptionKey::GetSupportedDefaultMode());
 
   // Required for the FE's Catalog
   ABORT_IF_ERROR(impala::LibCache::Init());

http://git-wip-us.apache.org/repos/asf/impala/blob/31d197ad/be/src/util/cpu-info.cc
----------------------------------------------------------------------
diff --git a/be/src/util/cpu-info.cc b/be/src/util/cpu-info.cc
index 1e3fcde..570e677 100644
--- a/be/src/util/cpu-info.cc
+++ b/be/src/util/cpu-info.cc
@@ -91,7 +91,7 @@ static struct {
   { "popcnt",    CpuInfo::POPCNT },
   { "avx",       CpuInfo::AVX },
   { "avx2",      CpuInfo::AVX2 },
-  { "pclmuldqd", CpuInfo::PCLMULQDQ }
+  { "pclmulqdq", CpuInfo::PCLMULQDQ }
 };
 static const long num_flags = sizeof(flag_mappings) / sizeof(flag_mappings[0]);
 

http://git-wip-us.apache.org/repos/asf/impala/blob/31d197ad/be/src/util/openssl-util.cc
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.cc b/be/src/util/openssl-util.cc
index 83cd8fd..2b368da 100644
--- a/be/src/util/openssl-util.cc
+++ b/be/src/util/openssl-util.cc
@@ -223,7 +223,7 @@ void EncryptionKey::SetCipherMode(AES_CIPHER_MODE m) {
   }
 }
 
-bool EncryptionKey::IsModeSupported(AES_CIPHER_MODE m) const {
+bool EncryptionKey::IsModeSupported(AES_CIPHER_MODE m) {
   switch (m) {
     case AES_256_GCM:
       // It becomes a bit tricky for GCM mode, because GCM mode is enabled since
@@ -251,13 +251,13 @@ bool EncryptionKey::IsModeSupported(AES_CIPHER_MODE m) const {
   }
 }
 
-AES_CIPHER_MODE EncryptionKey::GetSupportedDefaultMode() const {
+AES_CIPHER_MODE EncryptionKey::GetSupportedDefaultMode() {
   if (IsModeSupported(AES_256_GCM)) return AES_256_GCM;
   if (IsModeSupported(AES_256_CTR)) return AES_256_CTR;
   return AES_256_CFB;
 }
 
-const string EncryptionKey::ModeToString(AES_CIPHER_MODE m) const {
+const string EncryptionKey::ModeToString(AES_CIPHER_MODE m) {
   switch(m) {
     case AES_256_GCM: return "AES-GCM";
     case AES_256_CTR: return "AES-CTR";

http://git-wip-us.apache.org/repos/asf/impala/blob/31d197ad/be/src/util/openssl-util.h
----------------------------------------------------------------------
diff --git a/be/src/util/openssl-util.h b/be/src/util/openssl-util.h
index ef53425..d9f0da2 100644
--- a/be/src/util/openssl-util.h
+++ b/be/src/util/openssl-util.h
@@ -134,6 +134,15 @@ class EncryptionKey {
   /// If is GCM mode at runtime
   bool IsGcmMode() const { return mode_ == AES_256_GCM; }
 
+  /// Returns the a default mode which is supported at runtime. If GCM mode
+  /// is supported, return AES_256_GCM as the default. If GCM is not supported,
+  /// but CTR is still supported, return AES_256_CTR. When both GCM and
+  /// CTR modes are not supported, return AES_256_CFB.
+  static AES_CIPHER_MODE GetSupportedDefaultMode();
+
+  /// Converts mode type to string.
+  static const std::string ModeToString(AES_CIPHER_MODE m);
+
  private:
   /// Helper method that encrypts/decrypts if 'encrypt' is true/false respectively.
   /// A buffer of input data 'data' of length 'len' is encrypted/decrypted with this
@@ -144,16 +153,7 @@ class EncryptionKey {
       uint8_t* out) WARN_UNUSED_RESULT;
 
   /// Check if mode m is supported at runtime
-  bool IsModeSupported(AES_CIPHER_MODE m) const;
-
-  /// Returns the a default mode which is supported at runtime. If GCM mode
-  /// is supported, return AES_256_GCM as the default. If GCM is not supported,
-  /// but CTR is still supported, return AES_256_CTR. When both GCM and
-  /// CTR modes are not supported, return AES_256_CFB.
-  AES_CIPHER_MODE GetSupportedDefaultMode() const;
-
-  /// Converts mode type to string.
-  const string ModeToString(AES_CIPHER_MODE m) const;
+  static bool IsModeSupported(AES_CIPHER_MODE m);
 
   /// Track whether this key has been initialized, to avoid accidentally using
   /// uninitialized keys.


[05/15] impala git commit: IMPALA-4835: Part 3: switch I/O buffers to buffer pool

Posted by ta...@apache.org.
http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/disk-io-mgr-stress.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-stress.cc b/be/src/runtime/io/disk-io-mgr-stress.cc
index ba1ad92..3fd33de 100644
--- a/be/src/runtime/io/disk-io-mgr-stress.cc
+++ b/be/src/runtime/io/disk-io-mgr-stress.cc
@@ -19,6 +19,8 @@
 
 #include "runtime/io/disk-io-mgr-stress.h"
 
+#include "runtime/bufferpool/reservation-tracker.h"
+#include "runtime/exec-env.h"
 #include "runtime/io/request-context.h"
 #include "util/time.h"
 
@@ -27,21 +29,20 @@
 using namespace impala;
 using namespace impala::io;
 
-static const float ABORT_CHANCE = .10f;
-static const int MIN_READ_LEN = 1;
-static const int MAX_READ_LEN = 20;
+constexpr float DiskIoMgrStress::ABORT_CHANCE;
+const int DiskIoMgrStress::MIN_READ_LEN;
+const int DiskIoMgrStress::MAX_READ_LEN;
 
-static const int MIN_FILE_LEN = 10;
-static const int MAX_FILE_LEN = 1024;
+const int DiskIoMgrStress::MIN_FILE_LEN;
+const int DiskIoMgrStress::MAX_FILE_LEN;
 
 // Make sure this is between MIN/MAX FILE_LEN to test more cases
-static const int MIN_READ_BUFFER_SIZE = 64;
-static const int MAX_READ_BUFFER_SIZE = 128;
+const int DiskIoMgrStress::MIN_READ_BUFFER_SIZE;
+const int DiskIoMgrStress::MAX_READ_BUFFER_SIZE;
 
-// Maximum bytes to allocate per scan range.
-static const int MAX_BUFFER_BYTES_PER_SCAN_RANGE = MAX_READ_BUFFER_SIZE * 3;
+const int DiskIoMgrStress::MAX_BUFFER_BYTES_PER_SCAN_RANGE;
 
-static const int CANCEL_READER_PERIOD_MS = 20;  // in ms
+const int DiskIoMgrStress::CANCEL_READER_PERIOD_MS;
 
 static void CreateTempFile(const char* filename, const char* data) {
   FILE* file = fopen(filename, "w");
@@ -50,7 +51,7 @@ static void CreateTempFile(const char* filename, const char* data) {
   fclose(file);
 }
 
-string GenerateRandomData() {
+string DiskIoMgrStress::GenerateRandomData() {
   int rand_len = rand() % (MAX_FILE_LEN - MIN_FILE_LEN) + MIN_FILE_LEN;
   stringstream ss;
   for (int i = 0; i < rand_len; ++i) {
@@ -62,6 +63,8 @@ string GenerateRandomData() {
 
 struct DiskIoMgrStress::Client {
   boost::mutex lock;
+  /// Pool for objects that is cleared when the client is (re-)initialized in NewClient().
+  ObjectPool obj_pool;
   unique_ptr<RequestContext> reader;
   int file_idx;
   vector<ScanRange*> scan_ranges;
@@ -95,6 +98,7 @@ DiskIoMgrStress::DiskIoMgrStress(int num_disks, int num_threads_per_disk,
 
   clients_ = new Client[num_clients_];
   client_mem_trackers_.resize(num_clients_);
+  buffer_pool_clients_.reset(new BufferPool::ClientHandle[num_clients_]);
   for (int i = 0; i < num_clients_; ++i) {
     NewClient(i);
   }
@@ -119,8 +123,8 @@ void DiskIoMgrStress::ClientThread(int client_id) {
       CHECK(status.ok() || status.IsCancelled());
       if (range == NULL) break;
       if (needs_buffers) {
-        status = io_mgr_->AllocateBuffersForRange(
-            client->reader.get(), range, MAX_BUFFER_BYTES_PER_SCAN_RANGE);
+        status = io_mgr_->AllocateBuffersForRange(client->reader.get(),
+            &buffer_pool_clients_[client_id], range, MAX_BUFFER_BYTES_PER_SCAN_RANGE);
         CHECK(status.ok()) << status.GetDetail();
       }
 
@@ -212,7 +216,13 @@ void DiskIoMgrStress::Run(int sec) {
   }
   readers_.join_all();
 
-  for (unique_ptr<MemTracker>& mem_tracker : client_mem_trackers_) mem_tracker->Close();
+  for (int i = 0; i < num_clients_; ++i) {
+    if (clients_[i].reader != nullptr) {
+      io_mgr_->UnregisterContext(clients_[i].reader.get());
+    }
+    ExecEnv::GetInstance()->buffer_pool()->DeregisterClient(&buffer_pool_clients_[i]);
+    client_mem_trackers_[i]->Close();
+  }
   mem_tracker_.Close();
 }
 
@@ -234,26 +244,41 @@ void DiskIoMgrStress::NewClient(int i) {
     }
   }
 
-  for (int i = 0; i < client.scan_ranges.size(); ++i) {
-    delete client.scan_ranges[i];
-  }
+  // Clean up leftover state from the previous client (if any).
   client.scan_ranges.clear();
+  ExecEnv* exec_env = ExecEnv::GetInstance();
+  exec_env->buffer_pool()->DeregisterClient(&buffer_pool_clients_[i]);
+  if (client_mem_trackers_[i] != nullptr) client_mem_trackers_[i]->Close();
+  client.obj_pool.Clear();
 
   int assigned_len = 0;
   while (assigned_len < file_len) {
     int range_len = rand() % (MAX_READ_LEN - MIN_READ_LEN) + MIN_READ_LEN;
     range_len = min(range_len, file_len - assigned_len);
 
-    ScanRange* range = new ScanRange();
+    ScanRange* range = client.obj_pool.Add(new ScanRange);
     range->Reset(NULL, files_[client.file_idx].filename.c_str(), range_len, assigned_len,
         0, false, BufferOpts::Uncached());
     client.scan_ranges.push_back(range);
     assigned_len += range_len;
   }
 
-  if (client_mem_trackers_[i] != nullptr) client_mem_trackers_[i]->Close();
-  client_mem_trackers_[i].reset(new MemTracker(-1, "", &mem_tracker_));
-  client.reader = io_mgr_->RegisterContext(client_mem_trackers_[i].get());
-  Status status = io_mgr_->AddScanRanges(client.reader.get(), client.scan_ranges);
+  string client_name = Substitute("Client $0", i);
+  client_mem_trackers_[i].reset(new MemTracker(-1, client_name, &mem_tracker_));
+  Status status = exec_env->buffer_pool()->RegisterClient(client_name, nullptr,
+      exec_env->buffer_reservation(), client_mem_trackers_[i].get(),
+      numeric_limits<int64_t>::max(), RuntimeProfile::Create(&client.obj_pool, client_name),
+      &buffer_pool_clients_[i]);
+  CHECK(status.ok());
+  // Reserve enough memory for 3 buffers per range, which should be enough to guarantee
+  // progress.
+  CHECK(buffer_pool_clients_[i].IncreaseReservationToFit(
+      MAX_BUFFER_BYTES_PER_SCAN_RANGE * client.scan_ranges.size()))
+      << buffer_pool_clients_[i].DebugString() << "\n"
+      << exec_env->buffer_pool()->DebugString() << "\n"
+      << exec_env->buffer_reservation()->DebugString();
+
+  client.reader = io_mgr_->RegisterContext();
+  status = io_mgr_->AddScanRanges(client.reader.get(), client.scan_ranges);
   CHECK(status.ok());
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/disk-io-mgr-stress.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-stress.h b/be/src/runtime/io/disk-io-mgr-stress.h
index b872694..574b58c 100644
--- a/be/src/runtime/io/disk-io-mgr-stress.h
+++ b/be/src/runtime/io/disk-io-mgr-stress.h
@@ -22,8 +22,11 @@
 #include <memory>
 #include <vector>
 #include <boost/scoped_ptr.hpp>
+#include <boost/thread/condition_variable.hpp>
 #include <boost/thread/thread.hpp>
 
+#include "common/object-pool.h"
+#include "runtime/bufferpool/buffer-pool.h"
 #include "runtime/io/disk-io-mgr.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/thread-resource-mgr.h"
@@ -43,15 +46,29 @@ class DiskIoMgrStress {
   /// Run the test for 'sec'.  If 0, run forever
   void Run(int sec);
 
+  static constexpr float ABORT_CHANCE = .10f;
+  static const int MIN_READ_LEN = 1;
+  static const int MAX_READ_LEN = 20;
+
+  static const int MIN_FILE_LEN = 10;
+  static const int MAX_FILE_LEN = 1024;
+
+  // Make sure this is between MIN/MAX FILE_LEN to test more cases
+  static const int MIN_READ_BUFFER_SIZE = 64;
+  static const int MAX_READ_BUFFER_SIZE = 128;
+
+  // Maximum bytes to allocate per scan range.
+  static const int MAX_BUFFER_BYTES_PER_SCAN_RANGE = MAX_READ_BUFFER_SIZE * 3;
+
+  static const int CANCEL_READER_PERIOD_MS = 20;
  private:
   struct Client;
 
   struct File {
     std::string filename;
-    std::string data;  // the data in the file, used to validate
+    std::string data; // the data in the file, used to validate
   };
 
-
   /// Files used for testing.  These are created at startup and recycled
   /// during the test
   std::vector<File> files_;
@@ -72,6 +89,9 @@ class DiskIoMgrStress {
   /// Client MemTrackers, one per client.
   std::vector<std::unique_ptr<MemTracker>> client_mem_trackers_;
 
+  /// Buffer pool clients, one per client.
+  std::unique_ptr<BufferPool::ClientHandle[]> buffer_pool_clients_;
+
   /// If true, tests cancelling readers
   bool includes_cancellation_;
 
@@ -88,6 +108,8 @@ class DiskIoMgrStress {
 
   /// Possibly cancels a random reader.
   void CancelRandomReader();
+
+  static std::string GenerateRandomData();
 };
 }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/disk-io-mgr-test.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr-test.cc b/be/src/runtime/io/disk-io-mgr-test.cc
index 95ea184..3a0f727 100644
--- a/be/src/runtime/io/disk-io-mgr-test.cc
+++ b/be/src/runtime/io/disk-io-mgr-test.cc
@@ -22,14 +22,16 @@
 
 #include "codegen/llvm-codegen.h"
 #include "common/init.h"
+#include "runtime/bufferpool/buffer-pool.h"
+#include "runtime/bufferpool/reservation-tracker.h"
 #include "runtime/io/disk-io-mgr-stress.h"
 #include "runtime/io/disk-io-mgr.h"
 #include "runtime/io/request-context.h"
-#include "runtime/mem-tracker.h"
 #include "runtime/test-env.h"
 #include "runtime/thread-resource-mgr.h"
 #include "service/fe-support.h"
 #include "testutil/gtest-util.h"
+#include "testutil/rand-util.h"
 #include "util/condition-variable.h"
 #include "util/cpu-info.h"
 #include "util/disk-info.h"
@@ -37,13 +39,20 @@
 
 #include "common/names.h"
 
+using std::mt19937;
+using std::uniform_int_distribution;
+using std::uniform_real_distribution;
+
+DECLARE_int64(min_buffer_size);
 DECLARE_int32(num_remote_hdfs_io_threads);
 DECLARE_int32(num_s3_io_threads);
 DECLARE_int32(num_adls_io_threads);
 
 const int MIN_BUFFER_SIZE = 128;
 const int MAX_BUFFER_SIZE = 1024;
-const int LARGE_MEM_LIMIT = 1024 * 1024 * 1024;
+const int64_t LARGE_RESERVATION_LIMIT = 4L * 1024L * 1024L * 1024L;
+const int64_t LARGE_INITIAL_RESERVATION = 128L * 1024L * 1024L;
+const int64_t BUFFER_POOL_CAPACITY = LARGE_RESERVATION_LIMIT;
 
 namespace impala {
 namespace io {
@@ -53,15 +62,39 @@ class DiskIoMgrTest : public testing::Test {
 
   virtual void SetUp() {
     test_env_.reset(new TestEnv);
+    // Tests try to allocate arbitrarily small buffers. Ensure Buffer Pool allows it.
+    test_env_->SetBufferPoolArgs(1, BUFFER_POOL_CAPACITY);
     ASSERT_OK(test_env_->Init());
+    RandTestUtil::SeedRng("DISK_IO_MGR_TEST_SEED", &rng_);
   }
 
   virtual void TearDown() {
+    root_reservation_.Close();
     pool_.Clear();
+    test_env_.reset();
+  }
+
+  /// Initialises 'root_reservation_'. The reservation is automatically closed in
+  /// TearDown().
+  void InitRootReservation(int64_t reservation_limit) {
+    root_reservation_.InitRootTracker(
+        RuntimeProfile::Create(&pool_, "root"), reservation_limit);
+  }
+
+  /// Initialise 'client' with the given reservation limit. The client reservation is a
+  /// child of 'root_reservation_'.
+  void RegisterBufferPoolClient(int64_t reservation_limit, int64_t initial_reservation,
+      BufferPool::ClientHandle* client) {
+    ASSERT_OK(buffer_pool()->RegisterClient("", nullptr, &root_reservation_, nullptr,
+        reservation_limit, RuntimeProfile::Create(&pool_, ""), client));
+    if (initial_reservation > 0) {
+      ASSERT_TRUE(client->IncreaseReservation(initial_reservation));
+    }
   }
+
   void WriteValidateCallback(int num_writes, WriteRange** written_range,
-      DiskIoMgr* io_mgr, RequestContext* reader, int32_t* data,
-      Status expected_status, const Status& status) {
+      DiskIoMgr* io_mgr, RequestContext* reader, BufferPool::ClientHandle* client,
+      int32_t* data, Status expected_status, const Status& status) {
     if (expected_status.code() == TErrorCode::CANCELLED) {
       EXPECT_TRUE(status.ok() || status.IsCancelled()) << "Error: " << status.GetDetail();
     } else {
@@ -71,8 +104,8 @@ class DiskIoMgrTest : public testing::Test {
       ScanRange* scan_range = pool_.Add(new ScanRange());
       scan_range->Reset(nullptr, (*written_range)->file(), (*written_range)->len(),
           (*written_range)->offset(), 0, false, BufferOpts::Uncached());
-      ValidateSyncRead(io_mgr, reader, scan_range, reinterpret_cast<const char*>(data),
-          sizeof(int32_t));
+      ValidateSyncRead(io_mgr, reader, client, scan_range,
+          reinterpret_cast<const char*>(data), sizeof(int32_t));
     }
 
     {
@@ -93,9 +126,13 @@ class DiskIoMgrTest : public testing::Test {
 
  protected:
   void CreateTempFile(const char* filename, const char* data) {
+    CreateTempFile(filename, data, strlen(data));
+  }
+
+  void CreateTempFile(const char* filename, const char* data, int64_t data_bytes) {
     FILE* file = fopen(filename, "w");
     EXPECT_TRUE(file != nullptr);
-    fwrite(data, 1, strlen(data), file);
+    fwrite(data, 1, data_bytes, file);
     fclose(file);
   }
 
@@ -120,13 +157,14 @@ class DiskIoMgrTest : public testing::Test {
   }
 
   static void ValidateSyncRead(DiskIoMgr* io_mgr, RequestContext* reader,
-      ScanRange* range, const char* expected, int expected_len = -1) {
+      BufferPool::ClientHandle* client, ScanRange* range, const char* expected,
+      int expected_len = -1) {
     unique_ptr<BufferDescriptor> buffer;
     bool needs_buffers;
     ASSERT_OK(io_mgr->StartScanRange(reader, range, &needs_buffers));
     if (needs_buffers) {
       ASSERT_OK(io_mgr->AllocateBuffersForRange(
-          reader, range, io_mgr->max_buffer_size()));
+          reader, client, range, io_mgr->max_buffer_size()));
     }
     ASSERT_OK(range->GetNext(&buffer));
     ASSERT_TRUE(buffer != nullptr);
@@ -161,8 +199,8 @@ class DiskIoMgrTest : public testing::Test {
   // Continues pulling scan ranges from the io mgr until they are all done.
   // Updates num_ranges_processed with the number of ranges seen by this thread.
   static void ScanRangeThread(DiskIoMgr* io_mgr, RequestContext* reader,
-      const char* expected_result, int expected_len, const Status& expected_status,
-      int max_ranges, AtomicInt32* num_ranges_processed) {
+      BufferPool::ClientHandle* client, const char* expected_result, int expected_len,
+      const Status& expected_status, int max_ranges, AtomicInt32* num_ranges_processed) {
     int num_ranges = 0;
     while (max_ranges == 0 || num_ranges < max_ranges) {
       ScanRange* range;
@@ -172,7 +210,7 @@ class DiskIoMgrTest : public testing::Test {
       if (range == nullptr) break;
       if (needs_buffers) {
         ASSERT_OK(io_mgr->AllocateBuffersForRange(
-            reader, range, io_mgr->max_buffer_size() * 3));
+            reader, client, range, io_mgr->max_buffer_size() * 3));
       }
       ValidateScanRange(io_mgr, range, expected_result, expected_len, expected_status);
       num_ranges_processed->Add(1);
@@ -180,23 +218,27 @@ class DiskIoMgrTest : public testing::Test {
     }
   }
 
-  ScanRange* AllocateRange() {
-    return pool_.Add(new ScanRange);
-  }
-
-  ScanRange* InitRange(const char* file_path, int offset, int len,
+  ScanRange* InitRange(ObjectPool* pool, const char* file_path, int offset, int len,
       int disk_id, int64_t mtime, void* meta_data = nullptr, bool is_cached = false) {
-    ScanRange* range = AllocateRange();
+    ScanRange* range = pool->Add(new ScanRange);
     range->Reset(nullptr, file_path, len, offset, disk_id, true,
         BufferOpts(is_cached, mtime), meta_data);
     EXPECT_EQ(mtime, range->mtime());
     return range;
   }
 
+  /// Convenience function to get a reference to the buffer pool.
+  BufferPool* buffer_pool() const { return ExecEnv::GetInstance()->buffer_pool(); }
+
   boost::scoped_ptr<TestEnv> test_env_;
 
+  /// Per-test random number generator. Seeded before every test.
+  mt19937 rng_;
+
   ObjectPool pool_;
 
+  ReservationTracker root_reservation_;
+
   mutex written_mutex_;
   ConditionVariable writes_done_;
   int num_ranges_written_;
@@ -207,7 +249,7 @@ class DiskIoMgrTest : public testing::Test {
 // by reading the data back via a separate IoMgr instance. All writes are expected to
 // complete successfully.
 TEST_F(DiskIoMgrTest, SingleWriter) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   num_ranges_written_ = 0;
   string tmp_file = "/tmp/disk_io_mgr_test.txt";
   int num_ranges = 100;
@@ -221,24 +263,27 @@ TEST_F(DiskIoMgrTest, SingleWriter) {
   }
 
   scoped_ptr<DiskIoMgr> read_io_mgr(new DiskIoMgr(1, 1, 1, 1, 10));
-  MemTracker reader_mem_tracker(LARGE_MEM_LIMIT);
+  BufferPool::ClientHandle read_client;
+  RegisterBufferPoolClient(
+      LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
   ASSERT_OK(read_io_mgr->Init());
-  unique_ptr<RequestContext> reader = read_io_mgr->RegisterContext(&reader_mem_tracker);
+  unique_ptr<RequestContext> reader = read_io_mgr->RegisterContext();
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
-      pool_.Clear(); // Destroy scan ranges from previous iterations.
+      // Pool for temporary objects from this iteration only.
+      ObjectPool tmp_pool;
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 10);
       ASSERT_OK(io_mgr.Init());
-      unique_ptr<RequestContext> writer = io_mgr.RegisterContext(&mem_tracker);
+      unique_ptr<RequestContext> writer = io_mgr.RegisterContext();
       for (int i = 0; i < num_ranges; ++i) {
-        int32_t* data = pool_.Add(new int32_t);
+        int32_t* data = tmp_pool.Add(new int32_t);
         *data = rand();
-        WriteRange** new_range = pool_.Add(new WriteRange*);
-        WriteRange::WriteDoneCallback callback =
-            bind(mem_fn(&DiskIoMgrTest::WriteValidateCallback), this, num_ranges,
-                new_range, read_io_mgr.get(), reader.get(), data, Status::OK(), _1);
-        *new_range = pool_.Add(new WriteRange(
-            tmp_file, cur_offset, num_ranges % num_disks, callback));
+        WriteRange** new_range = tmp_pool.Add(new WriteRange*);
+        WriteRange::WriteDoneCallback callback = bind(
+            mem_fn(&DiskIoMgrTest::WriteValidateCallback), this, num_ranges, new_range,
+            read_io_mgr.get(), reader.get(), &read_client, data, Status::OK(), _1);
+        *new_range = tmp_pool.Add(
+            new WriteRange(tmp_file, cur_offset, num_ranges % num_disks, callback));
         (*new_range)->SetData(reinterpret_cast<uint8_t*>(data), sizeof(int32_t));
         EXPECT_OK(io_mgr.AddWriteRange(writer.get(), *new_range));
         cur_offset += sizeof(int32_t);
@@ -254,27 +299,26 @@ TEST_F(DiskIoMgrTest, SingleWriter) {
   }
 
   read_io_mgr->UnregisterContext(reader.get());
-  read_io_mgr.reset();
+  buffer_pool()->DeregisterClient(&read_client);
 }
 
 // Perform invalid writes (e.g. file in non-existent directory, negative offset) and
 // validate that an error status is returned via the write callback.
 TEST_F(DiskIoMgrTest, InvalidWrite) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   num_ranges_written_ = 0;
   string tmp_file = "/non-existent/file.txt";
   DiskIoMgr io_mgr(1, 1, 1, 1, 10);
   ASSERT_OK(io_mgr.Init());
-  unique_ptr<RequestContext> writer = io_mgr.RegisterContext(nullptr);
+  unique_ptr<RequestContext> writer = io_mgr.RegisterContext();
   int32_t* data = pool_.Add(new int32_t);
   *data = rand();
 
   // Write to file in non-existent directory.
   WriteRange** new_range = pool_.Add(new WriteRange*);
   WriteRange::WriteDoneCallback callback =
-      bind(mem_fn(&DiskIoMgrTest::WriteValidateCallback), this, 2, new_range,
-          (DiskIoMgr*)nullptr, (RequestContext*)nullptr, data,
-          Status(TErrorCode::DISK_IO_ERROR, "Test Failure"), _1);
+      bind(mem_fn(&DiskIoMgrTest::WriteValidateCallback), this, 2, new_range, nullptr,
+          nullptr, nullptr, data, Status(TErrorCode::DISK_IO_ERROR, "Test Failure"), _1);
   *new_range = pool_.Add(new WriteRange(tmp_file, rand(), 0, callback));
 
   (*new_range)->SetData(reinterpret_cast<uint8_t*>(data), sizeof(int32_t));
@@ -289,9 +333,9 @@ TEST_F(DiskIoMgrTest, InvalidWrite) {
   }
 
   new_range = pool_.Add(new WriteRange*);
-  callback = bind(mem_fn(&DiskIoMgrTest::WriteValidateCallback), this, 2,
-      new_range, (DiskIoMgr*)nullptr, (RequestContext*)nullptr,
-      data, Status(TErrorCode::DISK_IO_ERROR, "Test Failure"), _1);
+  callback = bind(mem_fn(&DiskIoMgrTest::WriteValidateCallback), this, 2, new_range,
+      nullptr, nullptr, nullptr, data,
+      Status(TErrorCode::DISK_IO_ERROR, "Test Failure"), _1);
 
   *new_range = pool_.Add(new WriteRange(tmp_file, -1, 0, callback));
   (*new_range)->SetData(reinterpret_cast<uint8_t*>(data), sizeof(int32_t));
@@ -309,7 +353,7 @@ TEST_F(DiskIoMgrTest, InvalidWrite) {
 // AddWriteRange() is expected to succeed before the cancel and fail after it.
 // The writes themselves may finish with status cancelled or ok.
 TEST_F(DiskIoMgrTest, SingleWriterCancel) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   num_ranges_written_ = 0;
   string tmp_file = "/tmp/disk_io_mgr_test.txt";
   int num_ranges = 100;
@@ -324,29 +368,33 @@ TEST_F(DiskIoMgrTest, SingleWriterCancel) {
   }
 
   scoped_ptr<DiskIoMgr> read_io_mgr(new DiskIoMgr(1, 1, 1, 1, 10));
-  MemTracker reader_mem_tracker(LARGE_MEM_LIMIT);
+  BufferPool::ClientHandle read_client;
+  RegisterBufferPoolClient(
+      LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
   ASSERT_OK(read_io_mgr->Init());
-  unique_ptr<RequestContext> reader = read_io_mgr->RegisterContext(&reader_mem_tracker);
+  unique_ptr<RequestContext> reader = read_io_mgr->RegisterContext();
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
-      pool_.Clear(); // Destroy scan ranges from previous iterations.
+      // Pool for temporary objects from this iteration only.
+      ObjectPool tmp_pool;
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 10);
       ASSERT_OK(io_mgr.Init());
-      unique_ptr<RequestContext> writer = io_mgr.RegisterContext(&mem_tracker);
+      unique_ptr<RequestContext> writer = io_mgr.RegisterContext();
       Status validate_status = Status::OK();
       for (int i = 0; i < num_ranges; ++i) {
         if (i == num_ranges_before_cancel) {
           writer->Cancel();
           validate_status = Status::CANCELLED;
         }
-        int32_t* data = pool_.Add(new int32_t);
+        int32_t* data = tmp_pool.Add(new int32_t);
         *data = rand();
-        WriteRange** new_range = pool_.Add(new WriteRange*);
-        WriteRange::WriteDoneCallback callback = bind(
-            mem_fn(&DiskIoMgrTest::WriteValidateCallback), this, num_ranges_before_cancel,
-            new_range, read_io_mgr.get(), reader.get(), data, Status::CANCELLED, _1);
-        *new_range = pool_.Add(new WriteRange(
-            tmp_file, cur_offset, num_ranges % num_disks, callback));
+        WriteRange** new_range = tmp_pool.Add(new WriteRange*);
+        WriteRange::WriteDoneCallback callback =
+            bind(mem_fn(&DiskIoMgrTest::WriteValidateCallback), this,
+                num_ranges_before_cancel, new_range, read_io_mgr.get(), reader.get(),
+                &read_client, data, Status::CANCELLED, _1);
+        *new_range = tmp_pool.Add(
+            new WriteRange(tmp_file, cur_offset, num_ranges % num_disks, callback));
         (*new_range)->SetData(reinterpret_cast<uint8_t*>(data), sizeof(int32_t));
         cur_offset += sizeof(int32_t);
         Status add_status = io_mgr.AddWriteRange(writer.get(), *new_range);
@@ -363,13 +411,13 @@ TEST_F(DiskIoMgrTest, SingleWriterCancel) {
   }
 
   read_io_mgr->UnregisterContext(reader.get());
-  read_io_mgr.reset();
+  buffer_pool()->DeregisterClient(&read_client);
 }
 
 // Basic test with a single reader, testing multiple threads, disks and a different
 // number of buffers.
 TEST_F(DiskIoMgrTest, SingleReader) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "abcdefghijklm";
   int len = strlen(data);
@@ -383,7 +431,7 @@ TEST_F(DiskIoMgrTest, SingleReader) {
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
       for (int num_read_threads = 1; num_read_threads <= 5; ++num_read_threads) {
-        ObjectPool pool;
+        ObjectPool tmp_pool;
         LOG(INFO) << "Starting test with num_threads_per_disk=" << num_threads_per_disk
                   << " num_disk=" << num_disks
                   << " num_read_threads=" << num_read_threads;
@@ -392,36 +440,39 @@ TEST_F(DiskIoMgrTest, SingleReader) {
         DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 1);
 
         ASSERT_OK(io_mgr.Init());
-        MemTracker reader_mem_tracker;
-        unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
+        BufferPool::ClientHandle read_client;
+        RegisterBufferPoolClient(
+            LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
+        unique_ptr<RequestContext> reader = io_mgr.RegisterContext();
 
         vector<ScanRange*> ranges;
         for (int i = 0; i < len; ++i) {
           int disk_id = i % num_disks;
-          ranges.push_back(InitRange(tmp_file, 0, len, disk_id, stat_val.st_mtime));
+          ranges.push_back(InitRange(&tmp_pool, tmp_file, 0, len, disk_id, stat_val.st_mtime));
         }
         ASSERT_OK(io_mgr.AddScanRanges(reader.get(), ranges));
 
         AtomicInt32 num_ranges_processed;
         thread_group threads;
         for (int i = 0; i < num_read_threads; ++i) {
-          threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(), data, len,
-              Status::OK(), 0, &num_ranges_processed));
+          threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(),
+              &read_client, data, len, Status::OK(), 0, &num_ranges_processed));
         }
         threads.join_all();
 
         EXPECT_EQ(num_ranges_processed.Load(), ranges.size());
         io_mgr.UnregisterContext(reader.get());
-        EXPECT_EQ(reader_mem_tracker.consumption(), 0);
+        EXPECT_EQ(read_client.GetUsedReservation(), 0);
+        buffer_pool()->DeregisterClient(&read_client);
       }
     }
   }
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
 }
 
 // This test issues adding additional scan ranges while there are some still in flight.
 TEST_F(DiskIoMgrTest, AddScanRangeTest) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "abcdefghijklm";
   int len = strlen(data);
@@ -434,7 +485,8 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) {
   int64_t iters = 0;
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
-      pool_.Clear(); // Destroy scan ranges from previous iterations.
+      // Pool for temporary objects from this iteration only.
+      ObjectPool tmp_pool;
       LOG(INFO) << "Starting test with num_threads_per_disk=" << num_threads_per_disk
                 << " num_disk=" << num_disks;
 
@@ -442,8 +494,10 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) {
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 1);
 
       ASSERT_OK(io_mgr.Init());
-      MemTracker reader_mem_tracker;
-      unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
+      BufferPool::ClientHandle read_client;
+      RegisterBufferPoolClient(
+          LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
+      unique_ptr<RequestContext> reader = io_mgr.RegisterContext();
 
       vector<ScanRange*> ranges_first_half;
       vector<ScanRange*> ranges_second_half;
@@ -451,10 +505,10 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) {
         int disk_id = i % num_disks;
         if (i > len / 2) {
           ranges_second_half.push_back(
-              InitRange(tmp_file, i, 1, disk_id, stat_val.st_mtime));
+              InitRange(&tmp_pool, tmp_file, i, 1, disk_id, stat_val.st_mtime));
         } else {
           ranges_first_half.push_back(
-              InitRange(tmp_file, i, 1, disk_id, stat_val.st_mtime));
+              InitRange(&tmp_pool, tmp_file, i, 1, disk_id, stat_val.st_mtime));
         }
       }
       AtomicInt32 num_ranges_processed;
@@ -463,8 +517,8 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) {
       ASSERT_OK(io_mgr.AddScanRanges(reader.get(), ranges_first_half));
 
       // Read a couple of them
-      ScanRangeThread(&io_mgr, reader.get(), data, strlen(data), Status::OK(), 2,
-          &num_ranges_processed);
+      ScanRangeThread(&io_mgr, reader.get(), &read_client, data, strlen(data),
+          Status::OK(), 2, &num_ranges_processed);
 
       // Issue second half
       ASSERT_OK(io_mgr.AddScanRanges(reader.get(), ranges_second_half));
@@ -472,24 +526,26 @@ TEST_F(DiskIoMgrTest, AddScanRangeTest) {
       // Start up some threads and then cancel
       thread_group threads;
       for (int i = 0; i < 3; ++i) {
-        threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(), data,
-            strlen(data), Status::CANCELLED, 0, &num_ranges_processed));
+        threads.add_thread(
+            new thread(ScanRangeThread, &io_mgr, reader.get(), &read_client, data,
+                strlen(data), Status::CANCELLED, 0, &num_ranges_processed));
       }
 
       threads.join_all();
       EXPECT_EQ(num_ranges_processed.Load(), len);
       io_mgr.UnregisterContext(reader.get());
-      EXPECT_EQ(reader_mem_tracker.consumption(), 0);
+      EXPECT_EQ(read_client.GetUsedReservation(), 0);
+      buffer_pool()->DeregisterClient(&read_client);
     }
   }
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
 }
 
 // Test to make sure that sync reads and async reads work together
 // Note: this test is constructed so the number of buffers is greater than the
 // number of scan ranges.
 TEST_F(DiskIoMgrTest, SyncReadTest) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "abcdefghijklm";
   int len = strlen(data);
@@ -502,7 +558,8 @@ TEST_F(DiskIoMgrTest, SyncReadTest) {
   int64_t iters = 0;
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
-      pool_.Clear(); // Destroy scan ranges from previous iterations.
+      // Pool for temporary objects from this iteration only.
+      ObjectPool tmp_pool;
       LOG(INFO) << "Starting test with num_threads_per_disk=" << num_threads_per_disk
                 << " num_disk=" << num_disks;
 
@@ -511,50 +568,55 @@ TEST_F(DiskIoMgrTest, SyncReadTest) {
           MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
 
       ASSERT_OK(io_mgr.Init());
-      MemTracker reader_mem_tracker;
-      unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
+      BufferPool::ClientHandle read_client;
+      RegisterBufferPoolClient(
+          LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
+      unique_ptr<RequestContext> reader = io_mgr.RegisterContext();
+
       ScanRange* complete_range =
-          InitRange(tmp_file, 0, strlen(data), 0, stat_val.st_mtime);
+          InitRange(&tmp_pool, tmp_file, 0, strlen(data), 0, stat_val.st_mtime);
 
       // Issue some reads before the async ones are issued
-      ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
-      ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
+      ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
+      ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
       vector<ScanRange*> ranges;
       for (int i = 0; i < len; ++i) {
         int disk_id = i % num_disks;
-        ranges.push_back(InitRange(tmp_file, 0, len, disk_id, stat_val.st_mtime));
+        ranges.push_back(
+            InitRange(&tmp_pool, tmp_file, 0, len, disk_id, stat_val.st_mtime));
       }
       ASSERT_OK(io_mgr.AddScanRanges(reader.get(), ranges));
 
       AtomicInt32 num_ranges_processed;
       thread_group threads;
       for (int i = 0; i < 5; ++i) {
-        threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(), data,
-            strlen(data), Status::OK(), 0, &num_ranges_processed));
+        threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(),
+            &read_client, data, strlen(data), Status::OK(), 0, &num_ranges_processed));
       }
 
       // Issue some more sync ranges
       for (int i = 0; i < 5; ++i) {
         sched_yield();
-        ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
+        ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
       }
 
       threads.join_all();
 
-      ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
-      ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
+      ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
+      ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
 
       EXPECT_EQ(num_ranges_processed.Load(), ranges.size());
       io_mgr.UnregisterContext(reader.get());
-      EXPECT_EQ(reader_mem_tracker.consumption(), 0);
+      EXPECT_EQ(read_client.GetUsedReservation(), 0);
+      buffer_pool()->DeregisterClient(&read_client);
     }
   }
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
 }
 
 // Tests a single reader cancelling half way through scan ranges.
 TEST_F(DiskIoMgrTest, SingleReaderCancel) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "abcdefghijklm";
   int len = strlen(data);
@@ -567,7 +629,8 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) {
   int64_t iters = 0;
   for (int num_threads_per_disk = 1; num_threads_per_disk <= 5; ++num_threads_per_disk) {
     for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
-      pool_.Clear(); // Destroy scan ranges from previous iterations.
+      // Pool for temporary objects from this iteration only.
+      ObjectPool tmp_pool;
       LOG(INFO) << "Starting test with num_threads_per_disk=" << num_threads_per_disk
                 << " num_disk=" << num_disks;
 
@@ -575,13 +638,16 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) {
       DiskIoMgr io_mgr(num_disks, num_threads_per_disk, num_threads_per_disk, 1, 1);
 
       ASSERT_OK(io_mgr.Init());
-      MemTracker reader_mem_tracker;
-      unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
+      BufferPool::ClientHandle read_client;
+      RegisterBufferPoolClient(
+          LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
+      unique_ptr<RequestContext> reader = io_mgr.RegisterContext();
 
       vector<ScanRange*> ranges;
       for (int i = 0; i < len; ++i) {
         int disk_id = i % num_disks;
-        ranges.push_back(InitRange(tmp_file, 0, len, disk_id, stat_val.st_mtime));
+        ranges.push_back(
+            InitRange(&tmp_pool, tmp_file, 0, len, disk_id, stat_val.st_mtime));
       }
       ASSERT_OK(io_mgr.AddScanRanges(reader.get(), ranges));
 
@@ -589,16 +655,17 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) {
       int num_succesful_ranges = ranges.size() / 2;
       // Read half the ranges
       for (int i = 0; i < num_succesful_ranges; ++i) {
-        ScanRangeThread(&io_mgr, reader.get(), data, strlen(data), Status::OK(), 1,
-            &num_ranges_processed);
+        ScanRangeThread(&io_mgr, reader.get(), &read_client, data, strlen(data),
+            Status::OK(), 1, &num_ranges_processed);
       }
       EXPECT_EQ(num_ranges_processed.Load(), num_succesful_ranges);
 
       // Start up some threads and then cancel
       thread_group threads;
       for (int i = 0; i < 3; ++i) {
-        threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(), data,
-            strlen(data), Status::CANCELLED, 0, &num_ranges_processed));
+        threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(),
+            &read_client, data, strlen(data), Status::CANCELLED, 0,
+            &num_ranges_processed));
       }
 
       reader->Cancel();
@@ -607,87 +674,93 @@ TEST_F(DiskIoMgrTest, SingleReaderCancel) {
       threads.join_all();
       EXPECT_TRUE(reader->IsCancelled());
       io_mgr.UnregisterContext(reader.get());
-      EXPECT_EQ(reader_mem_tracker.consumption(), 0);
+      EXPECT_EQ(read_client.GetUsedReservation(), 0);
+      buffer_pool()->DeregisterClient(&read_client);
     }
   }
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
 }
 
-// Test when the reader goes over the mem limit
-TEST_F(DiskIoMgrTest, MemLimits) {
+// Test readers running with different amounts of memory and getting blocked on scan
+// ranges that have run out of buffers.
+TEST_F(DiskIoMgrTest, MemScarcity) {
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
-  const char* data = "abcdefghijklm";
-  int len = strlen(data);
-  CreateTempFile(tmp_file, data);
+  // File is 2.5 max buffers so that we can scan file without returning buffers
+  // when we get the max reservation below.
+  const int64_t DATA_BYTES = MAX_BUFFER_SIZE * 5 / 2;
+  char data[DATA_BYTES];
+  for (int i = 0; i < DATA_BYTES; ++i) {
+    data[i] = uniform_int_distribution<uint8_t>(0, 255)(rng_);
+  }
+  CreateTempFile(tmp_file, data, DATA_BYTES);
 
   // Get mtime for file
   struct stat stat_val;
   stat(tmp_file, &stat_val);
 
-  const int mem_limit_num_buffers = 2;
-  // Allocate enough ranges so that the total buffers exceeds the mem limit.
+  const int RESERVATION_LIMIT_NUM_BUFFERS = 20;
+  const int64_t RESERVATION_LIMIT = RESERVATION_LIMIT_NUM_BUFFERS * MAX_BUFFER_SIZE;
+  InitRootReservation(RESERVATION_LIMIT);
+
+  thread_group threads;
+  // Allocate enough ranges so that the total buffers exceeds the limit.
   const int num_ranges = 25;
   {
-    MemTracker root_mem_tracker(mem_limit_num_buffers * MAX_BUFFER_SIZE);
     DiskIoMgr io_mgr(1, 1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
 
     ASSERT_OK(io_mgr.Init());
-    MemTracker reader_mem_tracker(-1, "Reader", &root_mem_tracker);
-    unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
+    BufferPool::ClientHandle read_client;
+    RegisterBufferPoolClient(RESERVATION_LIMIT, RESERVATION_LIMIT, &read_client);
+    unique_ptr<RequestContext> reader = io_mgr.RegisterContext();
 
     vector<ScanRange*> ranges;
     for (int i = 0; i < num_ranges; ++i) {
-      ranges.push_back(InitRange(tmp_file, 0, len, 0, stat_val.st_mtime));
+      ranges.push_back(InitRange(&pool_, tmp_file, 0, DATA_BYTES, 0, stat_val.st_mtime));
     }
     ASSERT_OK(io_mgr.AddScanRanges(reader.get(), ranges));
-
-    // Don't return buffers to force memory pressure
-    vector<pair<ScanRange*, unique_ptr<BufferDescriptor>>> buffers;
-
-    AtomicInt32 num_ranges_processed;
-    ScanRangeThread(&io_mgr, reader.get(), data, strlen(data), Status::MemLimitExceeded(),
-        1, &num_ranges_processed);
-
-    bool hit_mem_limit_exceeded = false;
-    char result[strlen(data) + 1];
-    // Keep starting new ranges without returning buffers. This forces us to go over
-    // the limit eventually.
-    while (true) {
-      memset(result, 0, strlen(data) + 1);
+    // Keep starting new ranges without returning buffers until we run out of
+    // reservation.
+    while (read_client.GetUnusedReservation() >= MIN_BUFFER_SIZE) {
       ScanRange* range = nullptr;
       bool needs_buffers;
-      Status status = io_mgr.GetNextUnstartedRange(reader.get(), &range, &needs_buffers);
-      ASSERT_TRUE(status.ok() || status.IsMemLimitExceeded());
-      hit_mem_limit_exceeded |= status.IsMemLimitExceeded();
+      ASSERT_OK(io_mgr.GetNextUnstartedRange(reader.get(), &range, &needs_buffers));
       if (range == nullptr) break;
-      DCHECK(needs_buffers);
-      status = io_mgr.AllocateBuffersForRange(reader.get(), range, MAX_BUFFER_SIZE * 3);
-      ASSERT_TRUE(status.ok() || status.IsMemLimitExceeded());
-      if (status.IsMemLimitExceeded()) {
-        hit_mem_limit_exceeded = true;
-        continue;
-      }
-
-      while (true) {
-        unique_ptr<BufferDescriptor> buffer;
-        Status status = range->GetNext(&buffer);
-        ASSERT_TRUE(status.ok() || status.IsMemLimitExceeded());
-        hit_mem_limit_exceeded |= status.IsMemLimitExceeded();
-        if (buffer == nullptr) break;
-        memcpy(result + range->offset() + buffer->scan_range_offset(),
-            buffer->buffer(), buffer->len());
-        buffers.emplace_back(range, move(buffer));
-      }
-      ValidateEmptyOrCorrect(data, result, strlen(data));
-    }
-
-    for (int i = 0; i < buffers.size(); ++i) {
-      buffers[i].first->ReturnBuffer(move(buffers[i].second));
+      ASSERT_TRUE(needs_buffers);
+      // Pick a random amount of memory to reserve.
+      int64_t max_bytes_to_alloc = uniform_int_distribution<int64_t>(MIN_BUFFER_SIZE,
+          min<int64_t>(read_client.GetUnusedReservation(), MAX_BUFFER_SIZE * 3))(rng_);
+      ASSERT_OK(io_mgr.AllocateBuffersForRange(
+          reader.get(), &read_client, range, max_bytes_to_alloc));
+      // Start a thread fetching from the range. The thread will either finish the
+      // range or be cancelled.
+      threads.add_thread(new thread([&data, DATA_BYTES, range] {
+        // Don't return buffers to force memory pressure.
+        vector<unique_ptr<BufferDescriptor>> buffers;
+        int64_t data_offset = 0;
+        Status status;
+        while (true) {
+          unique_ptr<BufferDescriptor> buffer;
+          status = range->GetNext(&buffer);
+          ASSERT_TRUE(status.ok() || status.IsCancelled()) << status.GetDetail();
+          if (status.IsCancelled() || buffer == nullptr) break;
+          EXPECT_EQ(0, memcmp(data + data_offset, buffer->buffer(), buffer->len()));
+          data_offset += buffer->len();
+          buffers.emplace_back(move(buffer));
+        }
+        if (status.ok()) ASSERT_EQ(DATA_BYTES, data_offset);
+        for (auto& buffer : buffers) range->ReturnBuffer(move(buffer));
+      }));
+      // Let the thread start running before starting the next.
+      SleepForMs(10);
     }
-
-    EXPECT_TRUE(hit_mem_limit_exceeded) << "Should have run out of memory";
+    // Let the threads run for a bit then cancel everything.
+    SleepForMs(500);
+    reader->Cancel();
+    // Wait until the threads have returned their buffers before unregistering.
+    threads.join_all();
     io_mgr.UnregisterContext(reader.get());
-    EXPECT_EQ(reader_mem_tracker.consumption(), 0);
+    EXPECT_EQ(read_client.GetUsedReservation(), 0);
+    buffer_pool()->DeregisterClient(&read_client);
   }
 }
 
@@ -696,7 +769,7 @@ TEST_F(DiskIoMgrTest, MemLimits) {
 // only tests the fallback mechanism.
 // TODO: we can fake the cached read path without HDFS
 TEST_F(DiskIoMgrTest, CachedReads) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "abcdefghijklm";
   int len = strlen(data);
@@ -711,51 +784,54 @@ TEST_F(DiskIoMgrTest, CachedReads) {
     DiskIoMgr io_mgr(num_disks, 1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
 
     ASSERT_OK(io_mgr.Init());
-    MemTracker reader_mem_tracker;
-    unique_ptr<RequestContext> reader = io_mgr.RegisterContext(&reader_mem_tracker);
+    BufferPool::ClientHandle read_client;
+    RegisterBufferPoolClient(
+        LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
+    unique_ptr<RequestContext> reader = io_mgr.RegisterContext();
 
     ScanRange* complete_range =
-        InitRange(tmp_file, 0, strlen(data), 0, stat_val.st_mtime, nullptr, true);
+        InitRange(&pool_, tmp_file, 0, strlen(data), 0, stat_val.st_mtime, nullptr, true);
 
     // Issue some reads before the async ones are issued
-    ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
-    ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
+    ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
+    ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
 
     vector<ScanRange*> ranges;
     for (int i = 0; i < len; ++i) {
       int disk_id = i % num_disks;
       ranges.push_back(
-          InitRange(tmp_file, 0, len, disk_id, stat_val.st_mtime, nullptr, true));
+          InitRange(&pool_, tmp_file, 0, len, disk_id, stat_val.st_mtime, nullptr, true));
     }
     ASSERT_OK(io_mgr.AddScanRanges(reader.get(), ranges));
 
     AtomicInt32 num_ranges_processed;
     thread_group threads;
     for (int i = 0; i < 5; ++i) {
-      threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(), data,
-          strlen(data), Status::OK(), 0, &num_ranges_processed));
+      threads.add_thread(new thread(ScanRangeThread, &io_mgr, reader.get(), &read_client,
+          data, strlen(data), Status::OK(), 0, &num_ranges_processed));
     }
 
     // Issue some more sync ranges
     for (int i = 0; i < 5; ++i) {
       sched_yield();
-      ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
+      ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
     }
 
     threads.join_all();
 
-    ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
-    ValidateSyncRead(&io_mgr, reader.get(), complete_range, data);
+    ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
+    ValidateSyncRead(&io_mgr, reader.get(), &read_client, complete_range, data);
 
     EXPECT_EQ(num_ranges_processed.Load(), ranges.size());
     io_mgr.UnregisterContext(reader.get());
-    EXPECT_EQ(reader_mem_tracker.consumption(), 0);
+    EXPECT_EQ(read_client.GetUsedReservation(), 0);
+    buffer_pool()->DeregisterClient(&read_client);
   }
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
 }
 
 TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const int ITERATIONS = 1;
   const char* data = "abcdefghijklmnopqrstuvwxyz";
   const int num_contexts = 5;
@@ -777,17 +853,22 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
 
   int64_t iters = 0;
   vector<unique_ptr<RequestContext>> contexts(num_contexts);
+  unique_ptr<BufferPool::ClientHandle[]> clients(
+      new BufferPool::ClientHandle[num_contexts]);
   Status status;
   for (int iteration = 0; iteration < ITERATIONS; ++iteration) {
     for (int threads_per_disk = 1; threads_per_disk <= 5; ++threads_per_disk) {
       for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
+        // Pool for temporary objects from this iteration only.
+        ObjectPool tmp_pool;
         DiskIoMgr io_mgr(num_disks, threads_per_disk, threads_per_disk, MIN_BUFFER_SIZE,
             MAX_BUFFER_SIZE);
         ASSERT_OK(io_mgr.Init());
         for (int file_index = 0; file_index < num_contexts; ++file_index) {
-          contexts[file_index] = io_mgr.RegisterContext(&mem_tracker);
+          RegisterBufferPoolClient(
+              LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &clients[file_index]);
+          contexts[file_index] = io_mgr.RegisterContext();
         }
-        pool_.Clear();
         int read_offset = 0;
         int write_offset = 0;
         while (read_offset < file_size) {
@@ -799,11 +880,11 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
             int num_scan_ranges = min<int>(num_reads_queued, write_offset - read_offset);
             for (int i = 0; i < num_scan_ranges; ++i) {
               ranges.push_back(InitRange(
-                  file_name.c_str(), read_offset, 1, i % num_disks, stat_val.st_mtime));
-              threads.add_thread(
-                  new thread(ScanRangeThread, &io_mgr, contexts[context_index].get(),
-                      reinterpret_cast<const char*>(data + (read_offset % strlen(data))),
-                      1, Status::OK(), num_scan_ranges, &num_ranges_processed));
+                  &tmp_pool, file_name.c_str(), read_offset, 1, i % num_disks, stat_val.st_mtime));
+              threads.add_thread(new thread(ScanRangeThread, &io_mgr,
+                  contexts[context_index].get(), &clients[context_index],
+                  reinterpret_cast<const char*>(data + (read_offset % strlen(data))),
+                  1, Status::OK(), num_scan_ranges, &num_ranges_processed));
               ++read_offset;
             }
 
@@ -813,7 +894,7 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
               WriteRange::WriteDoneCallback callback =
                   bind(mem_fn(&DiskIoMgrTest::WriteCompleteCallback),
                       this, num_write_ranges, _1);
-              WriteRange* new_range = pool_.Add(new WriteRange(
+              WriteRange* new_range = tmp_pool.Add(new WriteRange(
                   file_name, write_offset, i % num_disks, callback));
               new_range->SetData(
                   reinterpret_cast<const uint8_t*>(data + (write_offset % strlen(data))),
@@ -832,6 +913,7 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
         } // while (read_offset < file_size)
         for (int file_index = 0; file_index < num_contexts; ++file_index) {
           io_mgr.UnregisterContext(contexts[file_index].get());
+          buffer_pool()->DeregisterClient(&clients[file_index]);
         }
       } // for (int num_disks
     } // for (int threads_per_disk
@@ -840,23 +922,19 @@ TEST_F(DiskIoMgrTest, MultipleReaderWriter) {
 
 // This test will test multiple concurrent reads each reading a different file.
 TEST_F(DiskIoMgrTest, MultipleReader) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const int NUM_READERS = 5;
   const int DATA_LEN = 50;
   const int ITERATIONS = 25;
   const int NUM_THREADS_PER_READER = 3;
 
-  vector<string> file_names;
-  vector<int64_t> mtimes;
-  vector<string> data;
-  vector<unique_ptr<RequestContext>> readers;
-  vector<char*> results;
-
-  file_names.resize(NUM_READERS);
-  readers.resize(NUM_READERS);
-  mtimes.resize(NUM_READERS);
-  data.resize(NUM_READERS);
-  results.resize(NUM_READERS);
+  vector<string> file_names(NUM_READERS);
+  vector<int64_t> mtimes(NUM_READERS);
+  vector<string> data(NUM_READERS);
+  unique_ptr<BufferPool::ClientHandle[]> clients(
+      new BufferPool::ClientHandle[NUM_READERS]);
+  vector<unique_ptr<RequestContext>> readers(NUM_READERS);
+  vector<char*> results(NUM_READERS);
 
   // Initialize data for each reader.  The data will be
   // 'abcd...' for reader one, 'bcde...' for reader two (wrapping around at 'z')
@@ -887,7 +965,8 @@ TEST_F(DiskIoMgrTest, MultipleReader) {
   for (int iteration = 0; iteration < ITERATIONS; ++iteration) {
     for (int threads_per_disk = 1; threads_per_disk <= 5; ++threads_per_disk) {
       for (int num_disks = 1; num_disks <= 5; num_disks += 2) {
-        pool_.Clear(); // Destroy scan ranges from previous iterations.
+        // Pool for temporary objects from this iteration only.
+        ObjectPool tmp_pool;
         LOG(INFO) << "Starting test with num_threads_per_disk=" << threads_per_disk
                   << " num_disk=" << num_disks;
         if (++iters % 2500 == 0) LOG(ERROR) << "Starting iteration " << iters;
@@ -897,12 +976,14 @@ TEST_F(DiskIoMgrTest, MultipleReader) {
         ASSERT_OK(io_mgr.Init());
 
         for (int i = 0; i < NUM_READERS; ++i) {
-          readers[i] = io_mgr.RegisterContext(&mem_tracker);
+          RegisterBufferPoolClient(
+              LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &clients[i]);
+          readers[i] = io_mgr.RegisterContext();
 
           vector<ScanRange*> ranges;
           for (int j = 0; j < DATA_LEN; ++j) {
             int disk_id = j % num_disks;
-            ranges.push_back(InitRange(file_names[i].c_str(), j, 1, disk_id, mtimes[i]));
+            ranges.push_back(InitRange(&tmp_pool, file_names[i].c_str(), j, 1, disk_id, mtimes[i]));
           }
           ASSERT_OK(io_mgr.AddScanRanges(readers[i].get(), ranges));
         }
@@ -912,18 +993,20 @@ TEST_F(DiskIoMgrTest, MultipleReader) {
         for (int i = 0; i < NUM_READERS; ++i) {
           for (int j = 0; j < NUM_THREADS_PER_READER; ++j) {
             threads.add_thread(new thread(ScanRangeThread, &io_mgr, readers[i].get(),
-                data[i].c_str(), data[i].size(), Status::OK(), 0, &num_ranges_processed));
+                &clients[i], data[i].c_str(), data[i].size(), Status::OK(), 0,
+                &num_ranges_processed));
           }
         }
         threads.join_all();
         EXPECT_EQ(num_ranges_processed.Load(), DATA_LEN * NUM_READERS);
         for (int i = 0; i < NUM_READERS; ++i) {
           io_mgr.UnregisterContext(readers[i].get());
+          buffer_pool()->DeregisterClient(&clients[i]);
         }
       }
     }
   }
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
 }
 
 // Stress test for multiple clients with cancellation
@@ -937,7 +1020,7 @@ TEST_F(DiskIoMgrTest, StressTest) {
 
 // IMPALA-2366: handle partial read where range goes past end of file.
 TEST_F(DiskIoMgrTest, PartialRead) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "the quick brown fox jumped over the lazy dog";
   int len = strlen(data);
@@ -956,19 +1039,22 @@ TEST_F(DiskIoMgrTest, PartialRead) {
 
   for (int64_t max_buffer_size : max_buffer_sizes) {
     DiskIoMgr io_mgr(1, 1, 1, MIN_BUFFER_SIZE, max_buffer_size);
-
     ASSERT_OK(io_mgr.Init());
-    MemTracker reader_mem_tracker;
     unique_ptr<RequestContext> reader;
-    reader = io_mgr.RegisterContext(&reader_mem_tracker);
+    reader = io_mgr.RegisterContext();
+
+    BufferPool::ClientHandle read_client;
+    RegisterBufferPoolClient(
+        LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
 
     // We should not read past the end of file.
-    ScanRange* range = InitRange(tmp_file, 0, read_len, 0, stat_val.st_mtime);
+    ScanRange* range = InitRange(&pool_, tmp_file, 0, read_len, 0, stat_val.st_mtime);
     unique_ptr<BufferDescriptor> buffer;
     bool needs_buffers;
     ASSERT_OK(io_mgr.StartScanRange(reader.get(), range, &needs_buffers));
     if (needs_buffers) {
-      ASSERT_OK(io_mgr.AllocateBuffersForRange(reader.get(), range, 3 * max_buffer_size));
+      ASSERT_OK(io_mgr.AllocateBuffersForRange(
+          reader.get(), &read_client, range, 3 * max_buffer_size));
     }
 
     int64_t bytes_read = 0;
@@ -994,15 +1080,13 @@ TEST_F(DiskIoMgrTest, PartialRead) {
     } while (!eosr);
 
     io_mgr.UnregisterContext(reader.get());
-    EXPECT_EQ(reader_mem_tracker.consumption(), 0);
-    EXPECT_EQ(mem_tracker.consumption(), 0);
-    pool_.Clear();
+    EXPECT_EQ(read_client.GetUsedReservation(), 0);
+    buffer_pool()->DeregisterClient(&read_client);
   }
 }
 
 // Test zero-length scan range.
 TEST_F(DiskIoMgrTest, ZeroLengthScanRange) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "the quick brown fox jumped over the lazy dog";
   const int64_t MIN_BUFFER_SIZE = 2;
@@ -1016,12 +1100,9 @@ TEST_F(DiskIoMgrTest, ZeroLengthScanRange) {
   DiskIoMgr io_mgr(1, 1, 1, MIN_BUFFER_SIZE, MAX_BUFFER_SIZE);
 
   ASSERT_OK(io_mgr.Init());
-  MemTracker reader_mem_tracker;
-  unique_ptr<RequestContext> reader;
-  reader = io_mgr.RegisterContext(&reader_mem_tracker);
+  unique_ptr<RequestContext> reader = io_mgr.RegisterContext();
 
-  // We should not read past the end of file.
-  ScanRange* range = InitRange(tmp_file, 0, 0, 0, stat_val.st_mtime);
+  ScanRange* range = InitRange(&pool_, tmp_file, 0, 0, 0, stat_val.st_mtime);
   bool needs_buffers;
   Status status = io_mgr.StartScanRange(reader.get(), range, &needs_buffers);
   ASSERT_EQ(TErrorCode::DISK_IO_ERROR, status.code());
@@ -1035,7 +1116,6 @@ TEST_F(DiskIoMgrTest, ZeroLengthScanRange) {
 // Test what happens if don't call AllocateBuffersForRange() after trying to start a
 // range.
 TEST_F(DiskIoMgrTest, SkipAllocateBuffers) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "the quick brown fox jumped over the lazy dog";
   int len = strlen(data);
@@ -1051,13 +1131,12 @@ TEST_F(DiskIoMgrTest, SkipAllocateBuffers) {
 
   ASSERT_OK(io_mgr.Init());
   MemTracker reader_mem_tracker;
-  unique_ptr<RequestContext> reader;
-  reader = io_mgr.RegisterContext(&reader_mem_tracker);
+  unique_ptr<RequestContext> reader = io_mgr.RegisterContext();
 
   // We should not read past the end of file.
   vector<ScanRange*> ranges;
   for (int i = 0; i < 4; ++i) {
-    ranges.push_back(InitRange(tmp_file, 0, len, 0, stat_val.st_mtime));
+    ranges.push_back(InitRange(&pool_, tmp_file, 0, len, 0, stat_val.st_mtime));
   }
   bool needs_buffers;
   // Test StartScanRange().
@@ -1080,7 +1159,7 @@ TEST_F(DiskIoMgrTest, SkipAllocateBuffers) {
 
 // Test reading into a client-allocated buffer.
 TEST_F(DiskIoMgrTest, ReadIntoClientBuffer) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const char* tmp_file = "/tmp/disk_io_mgr_test.txt";
   const char* data = "the quick brown fox jumped over the lazy dog";
   int len = strlen(data);
@@ -1090,15 +1169,13 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBuffer) {
   scoped_ptr<DiskIoMgr> io_mgr(new DiskIoMgr(1, 1, 1, read_len, read_len));
 
   ASSERT_OK(io_mgr->Init());
-  // Reader doesn't need to provide mem tracker if it's providing buffers.
-  MemTracker* reader_mem_tracker = nullptr;
-  unique_ptr<RequestContext> reader;
-  reader = io_mgr->RegisterContext(reader_mem_tracker);
+  // Reader doesn't need to provide client if it's providing buffers.
+  unique_ptr<RequestContext> reader = io_mgr->RegisterContext();
 
   for (int buffer_len : vector<int>({len - 1, len, len + 1})) {
     vector<uint8_t> client_buffer(buffer_len);
     int scan_len = min(len, buffer_len);
-    ScanRange* range = AllocateRange();
+    ScanRange* range = pool_.Add(new ScanRange);
     range->Reset(nullptr, tmp_file, scan_len, 0, 0, true,
         BufferOpts::ReadInto(client_buffer.data(), buffer_len));
     bool needs_buffers;
@@ -1113,32 +1190,31 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBuffer) {
     ASSERT_EQ(memcmp(io_buffer->buffer(), data, scan_len), 0);
 
     // DiskIoMgr should not have allocated memory.
-    EXPECT_EQ(mem_tracker.consumption(), 0);
+    EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
     range->ReturnBuffer(move(io_buffer));
   }
 
   io_mgr->UnregisterContext(reader.get());
-  pool_.Clear();
-  io_mgr.reset();
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
 }
 
 // Test reading into a client-allocated buffer where the read fails.
 TEST_F(DiskIoMgrTest, ReadIntoClientBufferError) {
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const char* tmp_file = "/file/that/does/not/exist";
   const int SCAN_LEN = 128;
 
   scoped_ptr<DiskIoMgr> io_mgr(new DiskIoMgr(1, 1, 1, SCAN_LEN, SCAN_LEN));
 
   ASSERT_OK(io_mgr->Init());
-  // Reader doesn't need to provide mem tracker if it's providing buffers.
-  MemTracker* reader_mem_tracker = nullptr;
-  unique_ptr<RequestContext> reader;
   vector<uint8_t> client_buffer(SCAN_LEN);
   for (int i = 0; i < 1000; ++i) {
-    reader = io_mgr->RegisterContext(reader_mem_tracker);
-    ScanRange* range = AllocateRange();
+    // Reader doesn't need to provide mem tracker if it's providing buffers.
+    BufferPool::ClientHandle read_client;
+    RegisterBufferPoolClient(
+        LARGE_RESERVATION_LIMIT, LARGE_INITIAL_RESERVATION, &read_client);
+    unique_ptr<RequestContext> reader = io_mgr->RegisterContext();
+    ScanRange* range = pool_.Add(new ScanRange);
     range->Reset(nullptr, tmp_file, SCAN_LEN, 0, 0, true,
         BufferOpts::ReadInto(client_buffer.data(), SCAN_LEN));
     bool needs_buffers;
@@ -1153,25 +1229,25 @@ TEST_F(DiskIoMgrTest, ReadIntoClientBufferError) {
     ASSERT_FALSE(range->GetNext(&io_buffer).ok());
 
     // DiskIoMgr should not have allocated memory.
-    EXPECT_EQ(mem_tracker.consumption(), 0);
+    EXPECT_EQ(read_client.GetUsedReservation(), 0);
 
     io_mgr->UnregisterContext(reader.get());
+    EXPECT_EQ(read_client.GetUsedReservation(), 0);
+    buffer_pool()->DeregisterClient(&read_client);
   }
 
-  pool_.Clear();
-  io_mgr.reset();
-  EXPECT_EQ(mem_tracker.consumption(), 0);
+  EXPECT_EQ(root_reservation_.GetChildReservations(), 0);
 }
 
 // Test to verify configuration parameters for number of I/O threads per disk.
 TEST_F(DiskIoMgrTest, VerifyNumThreadsParameter) {
+  InitRootReservation(LARGE_RESERVATION_LIMIT);
   const int num_io_threads_for_remote_disks = FLAGS_num_remote_hdfs_io_threads
       + FLAGS_num_s3_io_threads + FLAGS_num_adls_io_threads;
 
   // Verify num_io_threads_per_rotational_disk and num_io_threads_per_solid_state_disk.
   // Since we do not have control over which disk is used, we check for either type
   // (rotational/solid state)
-  MemTracker mem_tracker(LARGE_MEM_LIMIT);
   const int num_io_threads_per_rotational_or_ssd = 2;
   DiskIoMgr io_mgr(1, num_io_threads_per_rotational_or_ssd,
       num_io_threads_per_rotational_or_ssd, 1, 10);

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/disk-io-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr.cc b/be/src/runtime/io/disk-io-mgr.cc
index 6dda447..6c7b9e6 100644
--- a/be/src/runtime/io/disk-io-mgr.cc
+++ b/be/src/runtime/io/disk-io-mgr.cc
@@ -15,8 +15,10 @@
 // specific language governing permissions and limitations
 // under the License.
 
-#include "common/global-flags.h"
 #include "runtime/io/disk-io-mgr.h"
+
+#include "common/global-flags.h"
+#include "runtime/exec-env.h"
 #include "runtime/io/disk-io-mgr-internal.h"
 #include "runtime/io/handle-cache.inline.h"
 
@@ -52,6 +54,8 @@ DEFINE_int32(num_threads_per_disk, 0, "Number of I/O threads per disk");
 static const int THREADS_PER_ROTATIONAL_DISK = 1;
 static const int THREADS_PER_SOLID_STATE_DISK = 8;
 
+const int64_t DiskIoMgr::IDEAL_MAX_SIZED_BUFFERS_PER_SCAN_RANGE;
+
 // The maximum number of the threads per rotational disk is also the max queue depth per
 // rotational disk.
 static const string num_io_threads_per_rotational_disk_help_msg = Substitute("Number of "
@@ -309,9 +313,8 @@ Status DiskIoMgr::Init() {
   return Status::OK();
 }
 
-unique_ptr<RequestContext> DiskIoMgr::RegisterContext(MemTracker* mem_tracker) {
-  return unique_ptr<RequestContext>(
-      new RequestContext(this, num_total_disks(), mem_tracker));
+unique_ptr<RequestContext> DiskIoMgr::RegisterContext() {
+  return unique_ptr<RequestContext>(new RequestContext(this, num_total_disks()));
 }
 
 void DiskIoMgr::UnregisterContext(RequestContext* reader) {
@@ -455,28 +458,21 @@ Status DiskIoMgr::GetNextUnstartedRange(RequestContext* reader, ScanRange** rang
   }
 }
 
-Status DiskIoMgr::AllocateBuffersForRange(RequestContext* reader, ScanRange* range,
-    int64_t max_bytes) {
+Status DiskIoMgr::AllocateBuffersForRange(RequestContext* reader,
+    BufferPool::ClientHandle* bp_client, ScanRange* range, int64_t max_bytes) {
   DCHECK_GE(max_bytes, min_buffer_size_);
   DCHECK(range->external_buffer_tag_ == ScanRange::ExternalBufferTag::NO_BUFFER)
      << static_cast<int>(range->external_buffer_tag_) << " invalid to allocate buffers "
      << "when already reading into an external buffer";
-
+  BufferPool* bp = ExecEnv::GetInstance()->buffer_pool();
   Status status;
   vector<unique_ptr<BufferDescriptor>> buffers;
   for (int64_t buffer_size : ChooseBufferSizes(range->len(), max_bytes)) {
-    if (!reader->mem_tracker_->TryConsume(buffer_size)) {
-      status = reader->mem_tracker_->MemLimitExceeded(nullptr,
-          "Failed to allocate I/O buffer", buffer_size);
-      goto error;
-    }
-    uint8_t* buffer = reinterpret_cast<uint8_t*>(malloc(buffer_size));
-    if (buffer == nullptr) {
-      reader->mem_tracker_->Release(buffer_size);
-      status = Status(Substitute("Failed to malloc $0-byte I/O buffer", buffer_size));
-      goto error;
-    }
-    buffers.emplace_back(new BufferDescriptor(this, reader, range, buffer, buffer_size));
+    BufferPool::BufferHandle handle;
+    status = bp->AllocateBuffer(bp_client, buffer_size, &handle);
+    if (!status.ok()) goto error;
+    buffers.emplace_back(new BufferDescriptor(
+        this, reader, range, bp_client, move(handle)));
   }
   range->AddUnusedBuffers(move(buffers), false);
   return Status::OK();

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/disk-io-mgr.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/disk-io-mgr.h b/be/src/runtime/io/disk-io-mgr.h
index d429d1d..d246e95 100644
--- a/be/src/runtime/io/disk-io-mgr.h
+++ b/be/src/runtime/io/disk-io-mgr.h
@@ -30,6 +30,7 @@
 #include "common/hdfs.h"
 #include "common/object-pool.h"
 #include "common/status.h"
+#include "runtime/bufferpool/buffer-pool.h"
 #include "runtime/io/handle-cache.h"
 #include "runtime/io/request-ranges.h"
 #include "runtime/thread-resource-mgr.h"
@@ -42,8 +43,6 @@
 
 namespace impala {
 
-class MemTracker;
-
 namespace io {
 /// Manager object that schedules IO for all queries on all disks and remote filesystems
 /// (such as S3). Each query maps to one or more RequestContext objects, each of which
@@ -163,6 +162,7 @@ namespace io {
 ///    buffer and one buffer is in the disk queue. The additional buffer can absorb
 ///    bursts where the producer runs faster than the consumer or the consumer runs
 ///    faster than the producer without blocking either the producer or consumer.
+/// See IDEAL_MAX_SIZED_BUFFERS_PER_SCAN_RANGE.
 ///
 /// Caching support:
 /// Scan ranges contain metadata on whether or not it is cached on the DN. In that
@@ -243,11 +243,7 @@ class DiskIoMgr : public CacheLineAligned {
   /// Allocates tracking structure for a request context.
   /// Register a new request context and return it to the caller. The caller must call
   /// UnregisterContext() for each context.
-  /// reader_mem_tracker: Is non-null only for readers. IO buffers
-  ///    used for this reader will be tracked by this. If the limit is exceeded
-  ///    the reader will be cancelled and MEM_LIMIT_EXCEEDED will be returned via
-  ///    GetNext().
-  std::unique_ptr<RequestContext> RegisterContext(MemTracker* reader_mem_tracker);
+  std::unique_ptr<RequestContext> RegisterContext();
 
   /// Unregisters context from the disk IoMgr by first cancelling it then blocking until
   /// all references to the context are removed from I/O manager internal data structures.
@@ -302,16 +298,15 @@ class DiskIoMgr : public CacheLineAligned {
   /// *needs_buffers=true.
   ///
   /// The buffer sizes are chosen based on range->len(). 'max_bytes' must be >=
-  /// min_read_buffer_size() so that at least one buffer can be allocated. Returns ok
-  /// if the buffers were successfully allocated and the range was scheduled. Fails with
-  /// MEM_LIMIT_EXCEEDED if the buffers could not be allocated. On failure, any allocated
-  /// buffers are freed and the state of 'range' is unmodified so that allocation can be
-  /// retried.  Setting 'max_bytes' to 3 * max_buffer_size() will typically maximize I/O
-  /// throughput. See Buffer management" section of the class comment for explanation.
-  /// TODO: error handling contract will change with reservations. The caller needs to
-  /// to guarantee that there is sufficient reservation.
-  Status AllocateBuffersForRange(RequestContext* reader, ScanRange* range,
-      int64_t max_bytes);
+  /// min_read_buffer_size() so that at least one buffer can be allocated. The caller
+  /// must ensure that 'bp_client' has at least 'max_bytes' unused reservation. Returns ok
+  /// if the buffers were successfully allocated and the range was scheduled.
+  ///
+  /// Setting 'max_bytes' to IDEAL_MAX_SIZED_BUFFERS_PER_SCAN_RANGE * max_buffer_size()
+  /// will typically maximize I/O throughput. See the "Buffer Management" section of
+  /// the class comment for explanation.
+  Status AllocateBuffersForRange(RequestContext* reader,
+      BufferPool::ClientHandle* bp_client, ScanRange* range, int64_t max_bytes);
 
   /// Determine which disk queue this file should be assigned to.  Returns an index into
   /// disk_queues_.  The disk_id is the volume ID for the local disk that holds the
@@ -379,6 +374,10 @@ class DiskIoMgr : public CacheLineAligned {
     REMOTE_NUM_DISKS
   };
 
+  /// The ideal number of max-sized buffers per scan range to maximise throughput.
+  /// See "Buffer Management" in the class comment for explanation.
+  static const int64_t IDEAL_MAX_SIZED_BUFFERS_PER_SCAN_RANGE = 3;
+
  private:
   friend class BufferDescriptor;
   friend class RequestContext;
@@ -401,7 +400,7 @@ class DiskIoMgr : public CacheLineAligned {
   /// Maximum read size. This is also the maximum size of each allocated buffer.
   const int64_t max_buffer_size_;
 
-  /// The minimum size of each read buffer.
+  /// The minimum size of each read buffer. Must be >= BufferPool::min_buffer_len().
   const int64_t min_buffer_size_;
 
   /// Thread group containing all the worker threads.

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/request-context.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-context.cc b/be/src/runtime/io/request-context.cc
index b124702..dec6aa6 100644
--- a/be/src/runtime/io/request-context.cc
+++ b/be/src/runtime/io/request-context.cc
@@ -17,6 +17,8 @@
 
 #include "runtime/io/disk-io-mgr-internal.h"
 
+#include "runtime/exec-env.h"
+
 #include "common/names.h"
 
 using namespace impala;
@@ -36,12 +38,28 @@ BufferDescriptor::BufferDescriptor(DiskIoMgr* io_mgr,
   DCHECK_GE(buffer_len, 0);
 }
 
+BufferDescriptor::BufferDescriptor(DiskIoMgr* io_mgr, RequestContext* reader,
+    ScanRange* scan_range, BufferPool::ClientHandle* bp_client,
+    BufferPool::BufferHandle handle) :
+  io_mgr_(io_mgr),
+  reader_(reader),
+  scan_range_(scan_range),
+  buffer_(handle.data()),
+  buffer_len_(handle.len()),
+  bp_client_(bp_client),
+  handle_(move(handle)) {
+  DCHECK(io_mgr != nullptr);
+  DCHECK(scan_range != nullptr);
+  DCHECK(bp_client_->is_registered());
+  DCHECK(handle_.is_open());
+}
+
 void RequestContext::FreeBuffer(BufferDescriptor* buffer) {
   DCHECK(buffer->buffer_ != nullptr);
   if (!buffer->is_cached() && !buffer->is_client_buffer()) {
-    // Only buffers that were not allocated by DiskIoMgr need to have memory freed.
-    free(buffer->buffer_);
-    mem_tracker_->Release(buffer->buffer_len_);
+    // Only buffers that were allocated by DiskIoMgr need to be freed.
+    ExecEnv::GetInstance()->buffer_pool()->FreeBuffer(
+        buffer->bp_client_, &buffer->handle_);
   }
   buffer->buffer_ = nullptr;
 }
@@ -200,9 +218,8 @@ void RequestContext::RemoveActiveScanRangeLocked(
   active_scan_ranges_.erase(range);
 }
 
-RequestContext::RequestContext(
-    DiskIoMgr* parent, int num_disks, MemTracker* tracker)
-  : parent_(parent), mem_tracker_(tracker), disk_states_(num_disks) {}
+RequestContext::RequestContext(DiskIoMgr* parent, int num_disks)
+  : parent_(parent), disk_states_(num_disks) {}
 
 // Dumps out request context information. Lock should be taken by caller
 string RequestContext::DebugString() const {

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/request-context.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-context.h b/be/src/runtime/io/request-context.h
index 3aea2bc..24fd0fc 100644
--- a/be/src/runtime/io/request-context.h
+++ b/be/src/runtime/io/request-context.h
@@ -158,12 +158,12 @@ class RequestContext {
     Inactive,
   };
 
-  RequestContext(DiskIoMgr* parent, int num_disks, MemTracker* tracker);
+  RequestContext(DiskIoMgr* parent, int num_disks);
 
-  /// Cleans up a buffer. If the buffer was allocated with AllocBuffer(), frees the buffer
-  /// memory and release the consumption to the client MemTracker. Otherwise (e.g. a
-  /// client or HDFS cache buffer), just prepares the descriptor to be destroyed.
-  /// After this is called, buffer->buffer() is NULL. Does not acquire 'lock_'.
+  /// Cleans up a buffer. If the buffer was allocated with AllocateBuffersForRange(),
+  /// frees the buffer. Otherwise (e.g. a client or HDFS cache buffer), just prepares the
+  /// descriptor to be destroyed. After this is called, buffer->buffer() is NULL.
+  /// Does not acquire 'lock_'.
   void FreeBuffer(BufferDescriptor* buffer);
 
   /// Decrements the number of active disks for this reader.  If the disk count
@@ -239,10 +239,6 @@ class RequestContext {
   /// Parent object
   DiskIoMgr* const parent_;
 
-  /// Memory used for this reader.  This is unowned by this object.
-  /// TODO: replace with bp client
-  MemTracker* const mem_tracker_;
-
   /// Total bytes read for this reader
   RuntimeProfile::Counter* bytes_read_counter_ = nullptr;
 

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/request-ranges.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/request-ranges.h b/be/src/runtime/io/request-ranges.h
index 041cb9d..0b234ac 100644
--- a/be/src/runtime/io/request-ranges.h
+++ b/be/src/runtime/io/request-ranges.h
@@ -32,8 +32,6 @@
 #include "util/mem-range.h"
 
 namespace impala {
-class MemTracker;
-
 namespace io {
 class DiskIoMgr;
 class RequestContext;
@@ -63,11 +61,15 @@ class BufferDescriptor {
   friend class ScanRange;
   friend class RequestContext;
 
-  /// Create a buffer descriptor for a new reader, range and data buffer. The buffer
-  /// memory should already be accounted against 'mem_tracker'.
+  /// Create a buffer descriptor for a new reader, range and data buffer.
   BufferDescriptor(DiskIoMgr* io_mgr, RequestContext* reader,
       ScanRange* scan_range, uint8_t* buffer, int64_t buffer_len);
 
+  /// Create a buffer descriptor allocated from the buffer pool.
+  BufferDescriptor(DiskIoMgr* io_mgr, RequestContext* reader,
+      ScanRange* scan_range, BufferPool::ClientHandle* bp_client,
+      BufferPool::BufferHandle handle);
+
   /// Return true if this is a cached buffer owned by HDFS.
   bool is_cached() const;
 
@@ -97,6 +99,11 @@ class BufferDescriptor {
   bool eosr_ = false;
 
   int64_t scan_range_offset_ = 0;
+
+  // Handle to an allocated buffer and the client used to allocate it buffer. Only used
+  // for non-external buffers.
+  BufferPool::ClientHandle* bp_client_ = nullptr;
+  BufferPool::BufferHandle handle_;
 };
 
 /// The request type, read or write associated with a request range.

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/io/scan-range.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/io/scan-range.cc b/be/src/runtime/io/scan-range.cc
index 0663a2b..9c2110c 100644
--- a/be/src/runtime/io/scan-range.cc
+++ b/be/src/runtime/io/scan-range.cc
@@ -340,9 +340,6 @@ void ScanRange::Reset(hdfsFS fs, const char* file, int64_t len, int64_t offset,
 void ScanRange::InitInternal(DiskIoMgr* io_mgr, RequestContext* reader) {
   DCHECK(exclusive_hdfs_fh_ == nullptr);
   DCHECK(local_file_ == nullptr);
-  // Reader must provide MemTracker or a buffer.
-  DCHECK(external_buffer_tag_ == ExternalBufferTag::CLIENT_BUFFER
-      || reader->mem_tracker_ != nullptr);
   io_mgr_ = io_mgr;
   reader_ = reader;
   local_file_ = nullptr;
@@ -650,8 +647,7 @@ Status ScanRange::ReadFromCache(
   }
 
   // Create a single buffer desc for the entire scan range and enqueue that.
-  // 'mem_tracker' is nullptr because the memory is owned by the HDFS java client,
-  // not the Impala backend.
+  // The memory is owned by the HDFS java client, not the Impala backend.
   unique_ptr<BufferDescriptor> desc = unique_ptr<BufferDescriptor>(new BufferDescriptor(
       io_mgr_, reader_, this, reinterpret_cast<uint8_t*>(buffer), 0));
   desc->len_ = bytes_read;

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/be/src/runtime/tmp-file-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/tmp-file-mgr.cc b/be/src/runtime/tmp-file-mgr.cc
index 3a69c33..e0c58d4 100644
--- a/be/src/runtime/tmp-file-mgr.cc
+++ b/be/src/runtime/tmp-file-mgr.cc
@@ -243,7 +243,7 @@ TmpFileMgr::FileGroup::FileGroup(TmpFileMgr* tmp_file_mgr, DiskIoMgr* io_mgr,
     next_allocation_index_(0),
     free_ranges_(64) {
   DCHECK(tmp_file_mgr != nullptr);
-  io_ctx_ = io_mgr_->RegisterContext(nullptr);
+  io_ctx_ = io_mgr_->RegisterContext();
 }
 
 TmpFileMgr::FileGroup::~FileGroup() {

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/common/thrift/PlanNodes.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/PlanNodes.thrift b/common/thrift/PlanNodes.thrift
index c5df1cd..1ab05a0 100644
--- a/common/thrift/PlanNodes.thrift
+++ b/common/thrift/PlanNodes.thrift
@@ -239,6 +239,9 @@ struct THdfsScanNode {
   // The byte offset of the slot for Parquet metadata if Parquet count star optimization
   // is enabled.
   10: optional i32 parquet_count_star_slot_offset
+
+  // The ideal memory reservation in bytes to process an input split.
+  11: optional i64 ideal_scan_range_reservation
 }
 
 struct TDataSourceScanNode {

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java b/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java
index 4f0a0e1..aae3863 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SlotDescriptor.java
@@ -163,6 +163,25 @@ public class SlotDescriptor {
   }
 
   /**
+   * Checks if this descriptor describes  an array "pos" pseudo-column.
+   *
+   * Note: checking whether the column is null distinguishes between top-level columns
+   * and nested types. This check more specifically looks just for a reference to the
+   * "pos" field of an array type.
+   */
+  public boolean isArrayPosRef() {
+    if (parent_ == null) return false;
+    Type parentType = parent_.getType();
+    if (parentType instanceof CollectionStructType) {
+      if (((CollectionStructType)parentType).isArrayStruct() &&
+          label_.equals(Path.ARRAY_POS_FIELD_NAME)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
    * Assembles the absolute materialized path to this slot starting from the schema
    * root. The materialized path points to the first non-struct schema element along the
    * path starting from the parent's tuple path to this slot's path.

http://git-wip-us.apache.org/repos/asf/impala/blob/8c922a6e/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/SlotRef.java b/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
index 23f2d88..0a945bd 100644
--- a/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
+++ b/fe/src/main/java/org/apache/impala/analysis/SlotRef.java
@@ -153,26 +153,6 @@ public class SlotRef extends Expr {
     return "<slot " + Integer.toString(desc_.getId().asInt()) + ">";
   }
 
-  /**
-   * Checks if this slotRef refers to an array "pos" pseudo-column.
-   *
-   * Note: checking whether the column is null distinguishes between top-level columns
-   * and nested types. This check more specifically looks just for a reference to the
-   * "pos" field of an array type.
-   */
-  public boolean isArrayPosRef() {
-    TupleDescriptor parent = getDesc().getParent();
-    if (parent == null) return false;
-    Type parentType = parent.getType();
-    if (parentType instanceof CollectionStructType) {
-      if (((CollectionStructType)parentType).isArrayStruct() &&
-          getDesc().getLabel().equals(Path.ARRAY_POS_FIELD_NAME)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
   @Override
   protected void toThrift(TExprNode msg) {
     msg.node_type = TExprNodeType.SLOT_REF;