You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by he...@apache.org on 2016/10/16 15:55:57 UTC

[1/7] incubator-impala git commit: IMPALA-2905: Handle coordinator fragment lifecycle like all others

Repository: incubator-impala
Updated Branches:
  refs/heads/master 05b91a973 -> 9f61397fc


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test b/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test
index 639f73c..e0d2d8c 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/with-clause.test
@@ -1,9 +1,13 @@
 # Basic test with a single with-clause view.
 with t as (select int_col x, bigint_col y from functional.alltypes) select x, y from t
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -13,9 +17,13 @@ with t as (select int_col x, bigint_col y from functional.alltypes) select x, y
 with t as (select int_col x, bigint_col y from functional.alltypes_view)
 select x, y from t
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -26,9 +34,13 @@ with t1 as (select int_col x, bigint_col y from functional.alltypes),
 t2 as (select 1 x , 10 y), t3 as (values(2 x , 20 y), (3, 30))
 select x, y from t2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=1
 ====
@@ -37,6 +49,8 @@ with t1 as (select int_col x, bigint_col y from functional.alltypes),
 t2 as (select 1 x , 10 y), t3 as (values(2 x , 20 y), (3, 30))
 select * from t1 union all select * from t2 union all select * from t3
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--03:UNION
@@ -48,6 +62,8 @@ select * from t1 union all select * from t2 union all select * from t3
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -71,6 +87,8 @@ t2 as (select int_col x, bigint_col y from functional.alltypestiny),
 t3 as (select int_col x, bigint_col y from functional.alltypessmall)
 select * from t1, t2, t3 where t1.x = t2.x and t2.x = t3.x
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: int_col = int_col
 |  runtime filters: RF000 <- int_col
@@ -90,6 +108,8 @@ select * from t1, t2, t3 where t1.x = t2.x and t2.x = t3.x
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> functional.alltypes.int_col, RF001 -> int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -134,6 +154,8 @@ where c1 > 0
 order by c3, c1 desc
 limit 3
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:TOP-N [LIMIT=3]
 |  order by: c3 ASC, c1 DESC
 |
@@ -155,6 +177,8 @@ limit 3
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:TOP-N [LIMIT=3]
 |  order by: c3 ASC, c1 DESC
 |
@@ -191,6 +215,8 @@ limit 3
 with t as (select int_col x, bigint_col y from functional.alltypestiny)
 select * from t t1 inner join t t2 on (t1.x = t2.x) inner join t t3 on (t2.x = t3.x)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: int_col = int_col
 |  runtime filters: RF000 <- int_col
@@ -210,6 +236,8 @@ select * from t t1 inner join t t2 on (t1.x = t2.x) inner join t t3 on (t2.x = t
    partitions=4/4 files=4 size=460B
    runtime filters: RF000 -> functional.alltypestiny.int_col, RF001 -> int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -242,6 +270,8 @@ select * from t t1 inner join t t2 on (t1.x = t2.x) inner join t t3 on (t2.x = t
 with t as (select int_col x, bigint_col y from functional.alltypestiny)
 select * from t t1 inner join t t2 using(x) inner join t t3 using(x)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: int_col = int_col
 |  runtime filters: RF000 <- int_col
@@ -261,6 +291,8 @@ select * from t t1 inner join t t2 using(x) inner join t t3 using(x)
    partitions=4/4 files=4 size=460B
    runtime filters: RF000 -> functional.alltypestiny.int_col, RF001 -> int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -293,6 +325,8 @@ select * from t t1 inner join t t2 using(x) inner join t t3 using(x)
 with t as (select int_col x, bigint_col y from functional.alltypestiny)
 select * from t t1 left outer join t t2 using(x) full outer join t t3 using(x)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: int_col = int_col
 |
@@ -308,6 +342,8 @@ select * from t t1 left outer join t t2 using(x) full outer join t t3 using(x)
 00:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
@@ -338,6 +374,8 @@ select * from t t1 left outer join t t2 using(x) full outer join t t3 using(x)
 with t as (select int_col x, bigint_col y from functional.alltypestiny)
 select * from t t1 inner join [broadcast] t t2 using(x) inner join [shuffle] t t3 using(x)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: int_col = int_col
 |  runtime filters: RF000 <- int_col
@@ -357,6 +395,8 @@ select * from t t1 inner join [broadcast] t t2 using(x) inner join [shuffle] t t
    partitions=4/4 files=4 size=460B
    runtime filters: RF000 -> functional.alltypestiny.int_col, RF001 -> int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -389,6 +429,8 @@ with t1 as (values('a', 'b'))
 (with t2 as (values('c', 'd')) select * from t2) union all
 (with t3 as (values('e', 'f')) select * from t3) order by 1 limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=1]
 |  order by: 'c' ASC
 |
@@ -400,6 +442,8 @@ with t1 as (values('a', 'b'))
 01:UNION
    constant-operands=1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=1]
 |  order by: 'c' ASC
 |
@@ -416,6 +460,8 @@ with t1 as (values('a', 'b'))
 (with t2 as (values('c', 'd')) select * from t2) union all
 (with t3 as (values('e', 'f')) select * from t3) order by 1 limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=1]
 |  order by: 'c' ASC
 |
@@ -427,6 +473,8 @@ with t1 as (values('a', 'b'))
 01:UNION
    constant-operands=1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=1]
 |  order by: 'c' ASC
 |
@@ -493,6 +541,8 @@ with t1 as (
   from functional.alltypestiny)
 select * from t1 where bigint_col = bigint_col2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
    predicates: bigint_col = bigint_col
@@ -503,6 +553,8 @@ select pos from functional.allcomplextypes t inner join
    select pos from w) v
 on v.pos = t.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [INNER JOIN]
@@ -520,6 +572,8 @@ select pos from functional.allcomplextypes t inner join
   (with w1 as (with w2 as (select pos from t.int_array_col) select * from w2)
    select pos from w1) v
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [CROSS JOIN]
@@ -536,6 +590,8 @@ with w1 as (select pos from functional.allcomplextypes t,
   (with w2 as (select pos from t.int_array_col) select * from w2) v)
 select * from w1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [CROSS JOIN]
@@ -553,6 +609,8 @@ select pos from functional.allcomplextypes t inner join
    w2 as (select key, value from t.map_map_col.value)
    select a1.*, m2.* from w1 a1, w1 a2, w2 m1, w2 m2) v on v.value = t.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--10:NESTED LOOP JOIN [INNER JOIN]

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/tests/custom_cluster/test_client_ssl.py
----------------------------------------------------------------------
diff --git a/tests/custom_cluster/test_client_ssl.py b/tests/custom_cluster/test_client_ssl.py
index f3ac523..4bc7bea 100644
--- a/tests/custom_cluster/test_client_ssl.py
+++ b/tests/custom_cluster/test_client_ssl.py
@@ -96,6 +96,7 @@ class TestClientSsl(CustomClusterTestSuite):
   @CustomClusterTestSuite.with_args(impalad_args=SSL_WILDCARD_ARGS,
                                     statestored_args=SSL_WILDCARD_ARGS,
                                     catalogd_args=SSL_WILDCARD_ARGS)
+  @pytest.mark.xfail(run=True, reason="IMPALA-4295 on Centos6")
   def test_wildcard_ssl(self, vector):
     """ Test for IMPALA-3159: Test with a certificate which has a wildcard for the
     CommonName.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/tests/failure/test_failpoints.py
----------------------------------------------------------------------
diff --git a/tests/failure/test_failpoints.py b/tests/failure/test_failpoints.py
index 1468f79..512d6f7 100644
--- a/tests/failure/test_failpoints.py
+++ b/tests/failure/test_failpoints.py
@@ -122,7 +122,7 @@ class TestFailpoints(ImpalaTestSuite):
 
     for node_id in node_ids:
       debug_action = '%d:%s:%s' % (node_id, location, FAILPOINT_ACTION_MAP[action])
-      LOG.info('Current dubug action: SET DEBUG_ACTION=%s' % debug_action)
+      LOG.info('Current debug action: SET DEBUG_ACTION=%s' % debug_action)
       vector.get_value('exec_option')['debug_action'] = debug_action
 
       if action == 'CANCEL':

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/tests/hs2/test_hs2.py
----------------------------------------------------------------------
diff --git a/tests/hs2/test_hs2.py b/tests/hs2/test_hs2.py
index 20bc9c7..6912b69 100644
--- a/tests/hs2/test_hs2.py
+++ b/tests/hs2/test_hs2.py
@@ -291,12 +291,16 @@ class TestHS2(HS2TestSuite):
     execute_statement_resp = self.hs2_client.ExecuteStatement(execute_statement_req)
     TestHS2.check_response(execute_statement_resp)
 
-    # Fetch results to make sure errors are generated
-    fetch_results_req = TCLIService.TFetchResultsReq()
-    fetch_results_req.operationHandle = execute_statement_resp.operationHandle
-    fetch_results_req.maxRows = 100
-    fetch_results_resp = self.hs2_client.FetchResults(fetch_results_req)
-    TestHS2.check_response(fetch_results_resp)
+    # Fetch results to make sure errors are generated. Errors are only guaranteed to be
+    # seen by the coordinator after FetchResults() returns eos.
+    has_more_results = True
+    while has_more_results:
+      fetch_results_req = TCLIService.TFetchResultsReq()
+      fetch_results_req.operationHandle = execute_statement_resp.operationHandle
+      fetch_results_req.maxRows = 100
+      fetch_results_resp = self.hs2_client.FetchResults(fetch_results_req)
+      TestHS2.check_response(fetch_results_resp)
+      has_more_results = fetch_results_resp.hasMoreRows
 
     get_log_req = TCLIService.TGetLogReq()
     get_log_req.operationHandle = execute_statement_resp.operationHandle

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/tests/hs2/test_json_endpoints.py
----------------------------------------------------------------------
diff --git a/tests/hs2/test_json_endpoints.py b/tests/hs2/test_json_endpoints.py
index 3053547..a5e73da 100644
--- a/tests/hs2/test_json_endpoints.py
+++ b/tests/hs2/test_json_endpoints.py
@@ -68,6 +68,10 @@ class TestJsonEndpoints(HS2TestSuite):
     fetch_results_req.maxRows = 100
     fetch_results_resp = self.hs2_client.FetchResults(fetch_results_req)
     TestJsonEndpoints.check_response(fetch_results_resp)
+    # Fetch one more time to ensure that query is at EOS (first fetch might return 0-size
+    # row batch)
+    fetch_results_resp = self.hs2_client.FetchResults(fetch_results_req)
+    TestJsonEndpoints.check_response(fetch_results_resp)
     queries_json = self._get_json_queries(http_addr)
     assert len(queries_json["in_flight_queries"]) == 1
     assert queries_json["num_in_flight_queries"] == 1

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/tests/shell/util.py
----------------------------------------------------------------------
diff --git a/tests/shell/util.py b/tests/shell/util.py
index cb0b3ea..4507706 100755
--- a/tests/shell/util.py
+++ b/tests/shell/util.py
@@ -91,9 +91,10 @@ def run_impala_shell_cmd(shell_args, expect_success=True, stdin_input=None):
   """
   result = run_impala_shell_cmd_no_expect(shell_args, stdin_input)
   if expect_success:
-    assert result.rc == 0, "Cmd %s was expected to succeed: %s" % (cmd, result.stderr)
+    assert result.rc == 0, "Cmd %s was expected to succeed: %s" % (shell_args,
+                                                                   result.stderr)
   else:
-    assert result.rc != 0, "Cmd %s was expected to fail" % cmd
+    assert result.rc != 0, "Cmd %s was expected to fail" % shell_args
   return result
 
 def run_impala_shell_cmd_no_expect(shell_args, stdin_input=None):


[6/7] incubator-impala git commit: IMPALA-2905: Handle coordinator fragment lifecycle like all others

Posted by he...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/runtime/plan-fragment-executor.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/plan-fragment-executor.cc b/be/src/runtime/plan-fragment-executor.cc
index aba4a26..5269fe5 100644
--- a/be/src/runtime/plan-fragment-executor.cc
+++ b/be/src/runtime/plan-fragment-executor.cc
@@ -26,22 +26,23 @@
 #include "common/logging.h"
 #include "common/object-pool.h"
 #include "exec/data-sink.h"
-#include "exec/exec-node.h"
 #include "exec/exchange-node.h"
-#include "exec/scan-node.h"
-#include "exec/hdfs-scan-node.h"
+#include "exec/exec-node.h"
 #include "exec/hbase-table-scanner.h"
+#include "exec/hdfs-scan-node.h"
+#include "exec/plan-root-sink.h"
+#include "exec/scan-node.h"
 #include "exprs/expr.h"
-#include "runtime/descriptors.h"
 #include "runtime/data-stream-mgr.h"
+#include "runtime/descriptors.h"
+#include "runtime/mem-tracker.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-filter-bank.h"
-#include "runtime/mem-tracker.h"
+#include "util/container-util.h"
 #include "util/cpu-info.h"
 #include "util/debug-util.h"
-#include "util/container-util.h"
-#include "util/parse-util.h"
 #include "util/mem-info.h"
+#include "util/parse-util.h"
 #include "util/periodic-counter-updater.h"
 #include "util/pretty-printer.h"
 
@@ -60,28 +61,45 @@ namespace impala {
 
 const string PlanFragmentExecutor::PER_HOST_PEAK_MEM_COUNTER = "PerHostPeakMemUsage";
 
-PlanFragmentExecutor::PlanFragmentExecutor(ExecEnv* exec_env,
-    const ReportStatusCallback& report_status_cb) :
-    exec_env_(exec_env), plan_(NULL), report_status_cb_(report_status_cb),
-    report_thread_active_(false), done_(false), closed_(false),
-    has_thread_token_(false), is_prepared_(false), is_cancelled_(false),
-    average_thread_tokens_(NULL), mem_usage_sampled_counter_(NULL),
-    thread_usage_sampled_counter_(NULL) {
-}
+PlanFragmentExecutor::PlanFragmentExecutor(
+    ExecEnv* exec_env, const ReportStatusCallback& report_status_cb)
+  : exec_env_(exec_env),
+    exec_tree_(NULL),
+    report_status_cb_(report_status_cb),
+    report_thread_active_(false),
+    closed_(false),
+    has_thread_token_(false),
+    is_prepared_(false),
+    is_cancelled_(false),
+    average_thread_tokens_(NULL),
+    mem_usage_sampled_counter_(NULL),
+    thread_usage_sampled_counter_(NULL) {}
 
 PlanFragmentExecutor::~PlanFragmentExecutor() {
-  Close();
+  DCHECK(!is_prepared_ || closed_);
   // at this point, the report thread should have been stopped
   DCHECK(!report_thread_active_);
 }
 
 Status PlanFragmentExecutor::Prepare(const TExecPlanFragmentParams& request) {
+  Status status = PrepareInternal(request);
+  prepared_promise_.Set(status);
+  return status;
+}
+
+Status PlanFragmentExecutor::WaitForOpen() {
+  DCHECK(prepared_promise_.IsSet()) << "Prepare() must complete before WaitForOpen()";
+  RETURN_IF_ERROR(prepared_promise_.Get());
+  return opened_promise_.Get();
+}
+
+Status PlanFragmentExecutor::PrepareInternal(const TExecPlanFragmentParams& request) {
   lock_guard<mutex> l(prepare_lock_);
   DCHECK(!is_prepared_);
 
   if (is_cancelled_) return Status::CANCELLED;
-
   is_prepared_ = true;
+
   // TODO: Break this method up.
   fragment_sw_.Start();
   const TPlanFragmentInstanceCtx& fragment_instance_ctx = request.fragment_instance_ctx;
@@ -100,6 +118,10 @@ Status PlanFragmentExecutor::Prepare(const TExecPlanFragmentParams& request) {
 
   // total_time_counter() is in the runtime_state_ so start it up now.
   SCOPED_TIMER(profile()->total_time_counter());
+  timings_profile_ =
+      obj_pool()->Add(new RuntimeProfile(obj_pool(), "PlanFragmentExecutor"));
+  profile()->AddChild(timings_profile_);
+  SCOPED_TIMER(ADD_TIMER(timings_profile_, "PrepareTime"));
 
   // reservation or a query option.
   int64_t bytes_limit = -1;
@@ -145,22 +167,22 @@ Status PlanFragmentExecutor::Prepare(const TExecPlanFragmentParams& request) {
 
   // set up plan
   DCHECK(request.__isset.fragment_ctx);
-  RETURN_IF_ERROR(ExecNode::CreateTree(runtime_state_.get(),
-      request.fragment_ctx.fragment.plan, *desc_tbl, &plan_));
-  runtime_state_->set_fragment_root_id(plan_->id());
+  RETURN_IF_ERROR(ExecNode::CreateTree(
+      runtime_state_.get(), request.fragment_ctx.fragment.plan, *desc_tbl, &exec_tree_));
+  runtime_state_->set_fragment_root_id(exec_tree_->id());
 
   if (fragment_instance_ctx.__isset.debug_node_id) {
     DCHECK(fragment_instance_ctx.__isset.debug_action);
     DCHECK(fragment_instance_ctx.__isset.debug_phase);
     ExecNode::SetDebugOptions(fragment_instance_ctx.debug_node_id,
-        fragment_instance_ctx.debug_phase, fragment_instance_ctx.debug_action, plan_);
+        fragment_instance_ctx.debug_phase, fragment_instance_ctx.debug_action,
+        exec_tree_);
   }
 
   // set #senders of exchange nodes before calling Prepare()
   vector<ExecNode*> exch_nodes;
-  plan_->CollectNodes(TPlanNodeType::EXCHANGE_NODE, &exch_nodes);
-  for (ExecNode* exch_node: exch_nodes)
-  {
+  exec_tree_->CollectNodes(TPlanNodeType::EXCHANGE_NODE, &exch_nodes);
+  for (ExecNode* exch_node : exch_nodes) {
     DCHECK_EQ(exch_node->type(), TPlanNodeType::EXCHANGE_NODE);
     int num_senders = FindWithDefault(fragment_instance_ctx.per_exch_num_senders,
         exch_node->id(), 0);
@@ -171,7 +193,7 @@ Status PlanFragmentExecutor::Prepare(const TExecPlanFragmentParams& request) {
   // set scan ranges
   vector<ExecNode*> scan_nodes;
   vector<TScanRangeParams> no_scan_ranges;
-  plan_->CollectScanNodes(&scan_nodes);
+  exec_tree_->CollectScanNodes(&scan_nodes);
   for (int i = 0; i < scan_nodes.size(); ++i) {
     ScanNode* scan_node = static_cast<ScanNode*>(scan_nodes[i]);
     const vector<TScanRangeParams>& scan_ranges = FindWithDefault(
@@ -179,42 +201,47 @@ Status PlanFragmentExecutor::Prepare(const TExecPlanFragmentParams& request) {
     scan_node->SetScanRanges(scan_ranges);
   }
 
-  RuntimeProfile::Counter* prepare_timer = ADD_TIMER(profile(), "PrepareTime");
+  RuntimeProfile::Counter* prepare_timer = ADD_TIMER(profile(), "ExecTreePrepareTime");
   {
     SCOPED_TIMER(prepare_timer);
-    RETURN_IF_ERROR(plan_->Prepare(runtime_state_.get()));
+    RETURN_IF_ERROR(exec_tree_->Prepare(runtime_state_.get()));
   }
 
   PrintVolumeIds(fragment_instance_ctx.per_node_scan_ranges);
 
-  // set up sink, if required
-  if (request.fragment_ctx.fragment.__isset.output_sink) {
-    RETURN_IF_ERROR(DataSink::CreateDataSink(
-        obj_pool(), request.fragment_ctx.fragment.output_sink,
-        request.fragment_ctx.fragment.output_exprs,
-        fragment_instance_ctx, row_desc(), &sink_));
-    sink_mem_tracker_.reset(new MemTracker(
-        -1, sink_->GetName(), runtime_state_->instance_mem_tracker(), true));
-    RETURN_IF_ERROR(sink_->Prepare(runtime_state(), sink_mem_tracker_.get()));
-
-    RuntimeProfile* sink_profile = sink_->profile();
-    if (sink_profile != NULL) {
-      profile()->AddChild(sink_profile);
-    }
-  } else {
-    sink_.reset(NULL);
+  DCHECK(request.fragment_ctx.fragment.__isset.output_sink);
+  RETURN_IF_ERROR(
+      DataSink::CreateDataSink(obj_pool(), request.fragment_ctx.fragment.output_sink,
+          request.fragment_ctx.fragment.output_exprs, fragment_instance_ctx,
+          exec_tree_->row_desc(), &sink_));
+  sink_mem_tracker_.reset(
+      new MemTracker(-1, sink_->GetName(), runtime_state_->instance_mem_tracker(), true));
+  RETURN_IF_ERROR(sink_->Prepare(runtime_state(), sink_mem_tracker_.get()));
+
+  RuntimeProfile* sink_profile = sink_->profile();
+  if (sink_profile != NULL) {
+    profile()->AddChild(sink_profile);
+  }
+
+  if (request.fragment_ctx.fragment.output_sink.type == TDataSinkType::PLAN_ROOT_SINK) {
+    root_sink_ = reinterpret_cast<PlanRootSink*>(sink_.get());
+    // Release the thread token on the root fragment instance. This fragment spends most
+    // of the time waiting and doing very little work. Holding on to the token causes
+    // underutilization of the machine. If there are 12 queries on this node, that's 12
+    // tokens reserved for no reason.
+    ReleaseThreadToken();
   }
 
   // set up profile counters
-  profile()->AddChild(plan_->runtime_profile());
+  profile()->AddChild(exec_tree_->runtime_profile());
   rows_produced_counter_ =
       ADD_COUNTER(profile(), "RowsProduced", TUnit::UNIT);
   per_host_mem_usage_ =
       ADD_COUNTER(profile(), PER_HOST_PEAK_MEM_COUNTER, TUnit::BYTES);
 
-  row_batch_.reset(new RowBatch(plan_->row_desc(), runtime_state_->batch_size(),
-        runtime_state_->instance_mem_tracker()));
-  VLOG(2) << "plan_root=\n" << plan_->DebugString();
+  row_batch_.reset(new RowBatch(exec_tree_->row_desc(), runtime_state_->batch_size(),
+      runtime_state_->instance_mem_tracker()));
+  VLOG(2) << "plan_root=\n" << exec_tree_->DebugString();
   return Status::OK();
 }
 
@@ -251,12 +278,21 @@ void PlanFragmentExecutor::PrintVolumeIds(
 }
 
 Status PlanFragmentExecutor::Open() {
-  VLOG_QUERY << "Open(): instance_id="
-      << runtime_state_->fragment_instance_id();
+  SCOPED_TIMER(profile()->total_time_counter());
+  SCOPED_TIMER(ADD_TIMER(timings_profile_, "OpenTime"));
+  VLOG_QUERY << "Open(): instance_id=" << runtime_state_->fragment_instance_id();
+  Status status = OpenInternal();
+  UpdateStatus(status);
+  opened_promise_.Set(status);
+  return status;
+}
 
-  RETURN_IF_ERROR(runtime_state_->desc_tbl().PrepareAndOpenPartitionExprs(runtime_state_.get()));
+Status PlanFragmentExecutor::OpenInternal() {
+  RETURN_IF_ERROR(
+      runtime_state_->desc_tbl().PrepareAndOpenPartitionExprs(runtime_state_.get()));
 
-  // we need to start the profile-reporting thread before calling Open(), since it
+  // we need to start the profile-reporting thread before calling exec_tree_->Open(),
+  // since it
   // may block
   if (!report_status_cb_.empty() && FLAGS_status_report_interval > 0) {
     unique_lock<mutex> l(report_thread_lock_);
@@ -271,22 +307,25 @@ Status PlanFragmentExecutor::Open() {
 
   OptimizeLlvmModule();
 
-  Status status = OpenInternal();
-  if (sink_.get() != NULL) {
-    // We call Close() here rather than in OpenInternal() because we want to make sure
-    // that Close() gets called even if there was an error in OpenInternal().
-    // We also want to call sink_->Close() here rather than in PlanFragmentExecutor::Close
-    // because we do not want the sink_ to hold on to all its resources as we will never
-    // use it after this.
-    sink_->Close(runtime_state());
-    // If there's a sink and no error, OpenInternal() completed the fragment execution.
-    if (status.ok()) {
-      done_ = true;
-      FragmentComplete();
-    }
+  {
+    SCOPED_TIMER(ADD_TIMER(timings_profile_, "ExecTreeOpenTime"));
+    RETURN_IF_ERROR(exec_tree_->Open(runtime_state_.get()));
   }
+  return sink_->Open(runtime_state_.get());
+}
 
-  if (!status.ok() && !status.IsCancelled() && !status.IsMemLimitExceeded()) {
+Status PlanFragmentExecutor::Exec() {
+  SCOPED_TIMER(ADD_TIMER(timings_profile_, "ExecTime"));
+  {
+    lock_guard<mutex> l(status_lock_);
+    RETURN_IF_ERROR(status_);
+  }
+  Status status = ExecInternal();
+
+  // If there's no error, ExecInternal() completed the fragment instance's execution.
+  if (status.ok()) {
+    FragmentComplete();
+  } else if (!status.IsCancelled() && !status.IsMemLimitExceeded()) {
     // Log error message in addition to returning in Status. Queries that do not
     // fetch results (e.g. insert) may not receive the message directly and can
     // only retrieve the log.
@@ -296,21 +335,23 @@ Status PlanFragmentExecutor::Open() {
   return status;
 }
 
-Status PlanFragmentExecutor::OpenInternal() {
-  SCOPED_TIMER(profile()->total_time_counter());
-  RETURN_IF_ERROR(plan_->Open(runtime_state_.get()));
-  if (sink_.get() == NULL) return Status::OK();
-
-  // If there is a sink, do all the work of driving it here, so that
-  // when this returns the query has actually finished
-  RETURN_IF_ERROR(sink_->Open(runtime_state_.get()));
-  while (!done_) {
+Status PlanFragmentExecutor::ExecInternal() {
+  RuntimeProfile::Counter* plan_exec_timer =
+      ADD_TIMER(timings_profile_, "ExecTreeExecTime");
+  bool exec_tree_complete = false;
+  do {
+    Status status;
     row_batch_->Reset();
-    RETURN_IF_ERROR(plan_->GetNext(runtime_state_.get(), row_batch_.get(), &done_));
-    if (VLOG_ROW_IS_ON) row_batch_->VLogRows("PlanFragmentExecutor::OpenInternal()");
+    {
+      SCOPED_TIMER(plan_exec_timer);
+      status = exec_tree_->GetNext(
+          runtime_state_.get(), row_batch_.get(), &exec_tree_complete);
+    }
+    if (VLOG_ROW_IS_ON) row_batch_->VLogRows("PlanFragmentExecutor::ExecInternal()");
     COUNTER_ADD(rows_produced_counter_, row_batch_->num_rows());
+    RETURN_IF_ERROR(status);
     RETURN_IF_ERROR(sink_->Send(runtime_state(), row_batch_.get()));
-  }
+  } while (!exec_tree_complete);
 
   // Flush the sink *before* stopping the report thread. Flush may need to add some
   // important information to the last report that gets sent. (e.g. table sinks record the
@@ -376,13 +417,20 @@ void PlanFragmentExecutor::SendReport(bool done) {
     status = status_;
   }
 
+  // If status is not OK, we need to make sure that only one sender sends a 'done'
+  // response.
+  // TODO: Clean all this up - move 'done' reporting to Close()?
+  if (!done && !status.ok()) {
+    done = completed_report_sent_.CompareAndSwap(0, 1);
+  }
+
   // Update the counter for the peak per host mem usage.
   per_host_mem_usage_->Set(runtime_state()->query_mem_tracker()->peak_consumption());
 
   // This will send a report even if we are cancelled.  If the query completed correctly
   // but fragments still need to be cancelled (e.g. limit reached), the coordinator will
   // be waiting for a final report and profile.
-  report_status_cb_(status, profile(), done || !status.ok());
+  report_status_cb_(status, profile(), done);
 }
 
 void PlanFragmentExecutor::StopReportThread() {
@@ -395,36 +443,6 @@ void PlanFragmentExecutor::StopReportThread() {
   report_thread_->Join();
 }
 
-Status PlanFragmentExecutor::GetNext(RowBatch** batch) {
-  SCOPED_TIMER(profile()->total_time_counter());
-  VLOG_FILE << "GetNext(): instance_id=" << runtime_state_->fragment_instance_id();
-
-  Status status = Status::OK();
-  row_batch_->Reset();
-  // Loop until we've got a non-empty batch, hit an error or exhausted the input.
-  while (!done_) {
-    status = plan_->GetNext(runtime_state_.get(), row_batch_.get(), &done_);
-    if (VLOG_ROW_IS_ON) row_batch_->VLogRows("PlanFragmentExecutor::GetNext()");
-    if (!status.ok()) break;
-    if (row_batch_->num_rows() > 0) break;
-    row_batch_->Reset();
-  }
-  UpdateStatus(status);
-  COUNTER_ADD(rows_produced_counter_, row_batch_->num_rows());
-
-  if (done_) {
-    VLOG_QUERY << "Finished executing fragment query_id=" << PrintId(query_id_)
-        << " instance_id=" << PrintId(runtime_state_->fragment_instance_id());
-    FragmentComplete();
-    // Once all rows are returned, signal that we're done with an empty batch.
-    *batch = row_batch_->num_rows() == 0 ? NULL : row_batch_.get();
-    return status;
-  }
-
-  *batch = row_batch_.get();
-  return status;
-}
-
 void PlanFragmentExecutor::FragmentComplete() {
   // Check the atomic flag. If it is set, then a fragment complete report has already
   // been sent.
@@ -463,7 +481,7 @@ void PlanFragmentExecutor::UpdateStatus(const Status& status) {
 }
 
 void PlanFragmentExecutor::Cancel() {
-  VLOG_QUERY << "Cancelling plan fragment...";
+  VLOG_QUERY << "Cancelling fragment instance...";
   lock_guard<mutex> l(prepare_lock_);
   is_cancelled_ = true;
   if (!is_prepared_) {
@@ -476,18 +494,10 @@ void PlanFragmentExecutor::Cancel() {
   runtime_state_->stream_mgr()->Cancel(runtime_state_->fragment_instance_id());
 }
 
-const RowDescriptor& PlanFragmentExecutor::row_desc() {
-  return plan_->row_desc();
-}
-
 RuntimeProfile* PlanFragmentExecutor::profile() {
   return runtime_state_->runtime_profile();
 }
 
-bool PlanFragmentExecutor::ReachedLimit() {
-  return plan_->ReachedLimit();
-}
-
 void PlanFragmentExecutor::ReleaseThreadToken() {
   if (has_thread_token_) {
     has_thread_token_ = false;
@@ -500,19 +510,23 @@ void PlanFragmentExecutor::ReleaseThreadToken() {
 
 void PlanFragmentExecutor::Close() {
   if (closed_) return;
+  if (!is_prepared_) return;
+  if (sink_.get() != nullptr) sink_->Close(runtime_state());
+
   row_batch_.reset();
   if (sink_mem_tracker_ != NULL) {
     sink_mem_tracker_->UnregisterFromParent();
     sink_mem_tracker_.reset();
   }
-  // Prepare may not have been called, which sets runtime_state_
-  if (runtime_state_.get() != NULL) {
-    if (plan_ != NULL) plan_->Close(runtime_state_.get());
-    runtime_state_->UnregisterReaderContexts();
-    exec_env_->thread_mgr()->UnregisterPool(runtime_state_->resource_pool());
-    runtime_state_->desc_tbl().ClosePartitionExprs(runtime_state_.get());
-    runtime_state_->filter_bank()->Close();
-  }
+
+  // Prepare should always have been called, and so runtime_state_ should be set
+  DCHECK(prepared_promise_.IsSet());
+  if (exec_tree_ != NULL) exec_tree_->Close(runtime_state_.get());
+  runtime_state_->UnregisterReaderContexts();
+  exec_env_->thread_mgr()->UnregisterPool(runtime_state_->resource_pool());
+  runtime_state_->desc_tbl().ClosePartitionExprs(runtime_state_.get());
+  runtime_state_->filter_bank()->Close();
+
   if (mem_usage_sampled_counter_ != NULL) {
     PeriodicCounterUpdater::StopTimeSeriesCounter(mem_usage_sampled_counter_);
     mem_usage_sampled_counter_ = NULL;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/runtime/plan-fragment-executor.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/plan-fragment-executor.h b/be/src/runtime/plan-fragment-executor.h
index f4355ea..82d3001 100644
--- a/be/src/runtime/plan-fragment-executor.h
+++ b/be/src/runtime/plan-fragment-executor.h
@@ -23,9 +23,10 @@
 #include <boost/scoped_ptr.hpp>
 #include <boost/function.hpp>
 
-#include "common/status.h"
 #include "common/object-pool.h"
+#include "common/status.h"
 #include "runtime/runtime-state.h"
+#include "util/promise.h"
 #include "util/runtime-profile-counters.h"
 #include "util/thread.h"
 
@@ -33,6 +34,7 @@ namespace impala {
 
 class HdfsFsCache;
 class ExecNode;
+class PlanRootSink;
 class RowDescriptor;
 class RowBatch;
 class DataSink;
@@ -45,21 +47,28 @@ class TPlanFragment;
 class TPlanExecParams;
 
 /// PlanFragmentExecutor handles all aspects of the execution of a single plan fragment,
-/// including setup and tear-down, both in the success and error case.
-/// Tear-down frees all memory allocated for this plan fragment and closes all data
-/// streams; it happens automatically in the d'tor.
-//
-/// The executor makes an aggregated profile for the entire fragment available,
-/// which includes profile information for the plan itself as well as the output
-/// sink, if any.
+/// including setup and tear-down, both in the success and error case. Tear-down, which
+/// happens in Close(), frees all memory allocated for this plan fragment and closes all
+/// data streams.
+///
+/// The lifecycle of a PlanFragmentExecutor is as follows:
+///     if (Prepare().ok()) {
+///       Open()
+///       Exec()
+///     }
+///     Close()
+///
+/// The executor makes an aggregated profile for the entire fragment available, which
+/// includes profile information for the plan itself as well as the output sink.
+///
 /// The ReportStatusCallback passed into the c'tor is invoked periodically to report the
 /// execution status. The frequency of those reports is controlled by the flag
 /// status_report_interval; setting that flag to 0 disables periodic reporting altogether
-/// Regardless of the value of that flag, if a report callback is specified, it is
-/// invoked at least once at the end of execution with an overall status and profile
-/// (and 'done' indicator). The only exception is when execution is cancelled, in which
-/// case the callback is *not* invoked (the coordinator already knows that execution
-/// stopped, because it initiated the cancellation).
+/// Regardless of the value of that flag, if a report callback is specified, it is invoked
+/// at least once at the end of execution with an overall status and profile (and 'done'
+/// indicator). The only exception is when execution is cancelled, in which case the
+/// callback is *not* invoked (the coordinator already knows that execution stopped,
+/// because it initiated the cancellation).
 //
 /// Aside from Cancel(), which may be called asynchronously, this class is not
 /// thread-safe.
@@ -76,49 +85,37 @@ class PlanFragmentExecutor {
       ReportStatusCallback;
 
   /// report_status_cb, if !empty(), is used to report the accumulated profile
-  /// information periodically during execution (Open() or GetNext()).
+  /// information periodically during execution.
   PlanFragmentExecutor(ExecEnv* exec_env, const ReportStatusCallback& report_status_cb);
 
-  /// Closes the underlying plan fragment and frees up all resources allocated
-  /// in Open()/GetNext().
-  /// It is an error to delete a PlanFragmentExecutor with a report callback
-  /// before Open()/GetNext() (depending on whether the fragment has a sink)
-  /// indicated that execution is finished.
+  /// It is an error to delete a PlanFragmentExecutor with a report callback before Exec()
+  /// indicated that execution is finished, or to delete one that has not been Close()'d
+  /// if Prepare() has been called.
   ~PlanFragmentExecutor();
 
   /// Prepare for execution. Call this prior to Open().
   ///
-  /// runtime_state() and row_desc() will not be valid until Prepare() is
-  /// called. runtime_state() will always be valid after Prepare() returns, unless the
-  /// query was cancelled before Prepare() was called.  If request.query_options.mem_limit
-  /// > 0, it is used as an approximate limit on the number of bytes this query can
-  /// consume at runtime.  The query will be aborted (MEM_LIMIT_EXCEEDED) if it goes over
-  /// that limit.
+  /// runtime_state() will not be valid until Prepare() is called. runtime_state() will
+  /// always be valid after Prepare() returns, unless the query was cancelled before
+  /// Prepare() was called.  If request.query_options.mem_limit > 0, it is used as an
+  /// approximate limit on the number of bytes this query can consume at runtime.  The
+  /// query will be aborted (MEM_LIMIT_EXCEEDED) if it goes over that limit.
   ///
   /// If Cancel() is called before Prepare(), Prepare() is a no-op and returns
   /// Status::CANCELLED;
   Status Prepare(const TExecPlanFragmentParams& request);
 
-  /// Start execution. Call this prior to GetNext().
-  /// If this fragment has a sink, Open() will send all rows produced
-  /// by the fragment to that sink. Therefore, Open() may block until
-  /// all rows are produced (and a subsequent call to GetNext() will not return
-  /// any rows).
-  /// This also starts the status-reporting thread, if the interval flag
-  /// is > 0 and a callback was specified in the c'tor.
-  /// If this fragment has a sink, report_status_cb will have been called for the final
-  /// time when Open() returns, and the status-reporting thread will have been stopped.
+  /// Opens the fragment plan and sink. Starts the profile reporting thread, if required.
   Status Open();
 
-  /// Return results through 'batch'. Sets '*batch' to NULL if no more results.
-  /// '*batch' is owned by PlanFragmentExecutor and must not be deleted.
-  /// When *batch == NULL, GetNext() should not be called anymore. Also, report_status_cb
-  /// will have been called for the final time and the status-reporting thread
-  /// will have been stopped.
-  Status GetNext(RowBatch** batch);
+  /// Executes the fragment by repeatedly driving the sink with batches produced by the
+  /// exec node tree. report_status_cb will have been called for the final time when
+  /// Exec() returns, and the status-reporting thread will have been stopped.
+  Status Exec();
 
-  /// Closes the underlying plan fragment and frees up all resources allocated
-  /// in Open()/GetNext().
+  /// Closes the underlying plan fragment and frees up all resources allocated in
+  /// Prepare() and Open(). Must be called if Prepare() has been called - no matter
+  /// whether or not Prepare() succeeded.
   void Close();
 
   /// Initiate cancellation. If called concurrently with Prepare(), will wait for
@@ -131,25 +128,30 @@ class PlanFragmentExecutor {
   /// It is legal to call Cancel() if Prepare() returned an error.
   void Cancel();
 
-  /// Returns true if this query has a limit and it has been reached.
-  bool ReachedLimit();
-
-  /// Releases the thread token for this fragment executor.
-  void ReleaseThreadToken();
-
   /// call these only after Prepare()
   RuntimeState* runtime_state() { return runtime_state_.get(); }
-  const RowDescriptor& row_desc();
 
   /// Profile information for plan and output sink.
   RuntimeProfile* profile();
 
+  /// Blocks until Prepare() is completed.
+  Status WaitForPrepare() { return prepared_promise_.Get(); }
+
+  /// Blocks until exec tree and sink are both opened. It is an error to call this before
+  /// Prepare() has completed. If Prepare() returned an error, WaitForOpen() will
+  /// return that error without blocking.
+  Status WaitForOpen();
+
+  /// Returns fragment instance's sink if this is the root fragment instance. Valid after
+  /// Prepare() returns; if Prepare() fails may be nullptr.
+  PlanRootSink* root_sink() { return root_sink_; }
+
   /// Name of the counter that is tracking per query, per host peak mem usage.
   static const std::string PER_HOST_PEAK_MEM_COUNTER;
 
  private:
   ExecEnv* exec_env_;  // not owned
-  ExecNode* plan_;  // lives in runtime_state_->obj_pool()
+  ExecNode* exec_tree_; // lives in runtime_state_->obj_pool()
   TUniqueId query_id_;
 
   /// profile reporting-related
@@ -166,9 +168,6 @@ class PlanFragmentExecutor {
   boost::condition_variable report_thread_started_cv_;
   bool report_thread_active_;  // true if we started the thread
 
-  /// true if plan_->GetNext() indicated that it's done
-  bool done_;
-
   /// true if Close() has been called
   bool closed_;
 
@@ -190,14 +189,20 @@ class PlanFragmentExecutor {
   /// (e.g. mem_trackers_) from 'runtime_state_' to 'sink_' need to be severed prior to
   /// the dtor of 'runtime_state_'.
   boost::scoped_ptr<RuntimeState> runtime_state_;
-  /// Output sink for rows sent to this fragment. May not be set, in which case rows are
-  /// returned via GetNext's row batch
-  /// Created in Prepare (if required), owned by this object.
+
+  /// Profile for timings for each stage of the plan fragment instance's lifecycle.
+  RuntimeProfile* timings_profile_;
+
+  /// Output sink for rows sent to this fragment. Created in Prepare(), owned by this
+  /// object.
   boost::scoped_ptr<DataSink> sink_;
   boost::scoped_ptr<MemTracker> sink_mem_tracker_;
 
+  /// Set if this fragment instance is the root of the entire plan, so that a consumer can
+  /// pull results by calling root_sink_->GetNext(). Same object as sink_.
+  PlanRootSink* root_sink_ = nullptr;
+
   boost::scoped_ptr<RowBatch> row_batch_;
-  boost::scoped_ptr<TRowBatch> thrift_batch_;
 
   /// Protects is_prepared_ and is_cancelled_, and is also used to coordinate between
   /// Prepare() and Cancel() to ensure mutual exclusion.
@@ -207,6 +212,12 @@ class PlanFragmentExecutor {
   /// error. If Cancel() was called before Prepare(), is_prepared_ will not be set.
   bool is_prepared_;
 
+  /// Set when Prepare() returns.
+  Promise<Status> prepared_promise_;
+
+  /// Set when OpenInternal() returns.
+  Promise<Status> opened_promise_;
+
   /// True if and only if Cancel() has been called.
   bool is_cancelled_;
 
@@ -267,21 +278,25 @@ class PlanFragmentExecutor {
   void FragmentComplete();
 
   /// Optimizes the code-generated functions in runtime_state_->llvm_codegen().
-  /// Must be called between plan_->Prepare() and plan_->Open().
-  /// This is somewhat time consuming so we don't want it to do it in
-  /// PlanFragmentExecutor()::Prepare() to allow starting plan fragments more
-  /// quickly and in parallel (in a deep plan tree, the fragments are started
-  /// in level order).
+  /// Must be called after exec_tree_->Prepare() and before exec_tree_->Open().
   void OptimizeLlvmModule();
 
   /// Executes Open() logic and returns resulting status. Does not set status_.
-  /// If this plan fragment has no sink, OpenInternal() does nothing.
-  /// If this plan fragment has a sink and OpenInternal() returns without an
-  /// error condition, all rows will have been sent to the sink, the sink will
-  /// have been closed, a final report will have been sent and the report thread will
-  /// have been stopped. sink_ will be set to NULL after successful execution.
   Status OpenInternal();
 
+  /// Pulls row batches from fragment instance and pushes them to sink_ in a loop. Returns
+  /// OK if the input was exhausted and sent to the sink successfully, an error otherwise.
+  /// If ExecInternal() returns without an error condition, all rows will have been sent
+  /// to the sink, the sink will have been closed, a final report will have been sent and
+  /// the report thread will have been stopped.
+  Status ExecInternal();
+
+  /// Performs all the logic of Prepare() and returns resulting status.
+  Status PrepareInternal(const TExecPlanFragmentParams& request);
+
+  /// Releases the thread token for this fragment executor.
+  void ReleaseThreadToken();
+
   /// Stops report thread, if one is running. Blocks until report thread terminates.
   /// Idempotent.
   void StopReportThread();

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/scheduling/query-schedule.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-schedule.cc b/be/src/scheduling/query-schedule.cc
index 5ad84df..1eb36e3 100644
--- a/be/src/scheduling/query-schedule.cc
+++ b/be/src/scheduling/query-schedule.cc
@@ -198,36 +198,28 @@ const TPlanFragment& FInstanceExecParams::fragment() const {
 }
 
 int QuerySchedule::GetNumFragmentInstances() const {
-  if (mt_fragment_exec_params_.empty()) return num_fragment_instances_;
   int result = 0;
-  for (const MtFragmentExecParams& fragment_exec_params: mt_fragment_exec_params_) {
-    result += fragment_exec_params.instance_exec_params.size();
+  if (mt_fragment_exec_params_.empty()) {
+    DCHECK(!fragment_exec_params_.empty());
+    for (const FragmentExecParams& fragment_exec_params : fragment_exec_params_) {
+      result += fragment_exec_params.hosts.size();
+    }
+  } else {
+    for (const MtFragmentExecParams& fragment_exec_params : mt_fragment_exec_params_) {
+      result += fragment_exec_params.instance_exec_params.size();
+    }
   }
   return result;
 }
 
-int QuerySchedule::GetNumRemoteFInstances() const {
-  bool has_coordinator_fragment = GetCoordFragment() != nullptr;
-  int result = GetNumFragmentInstances();
-  bool is_mt_execution = request_.query_ctx.request.query_options.mt_dop > 0;
-  if (is_mt_execution && has_coordinator_fragment) --result;
-  return result;
-}
-
-int QuerySchedule::GetTotalFInstances() const {
-  int result = GetNumRemoteFInstances();
-  return GetCoordFragment() != nullptr ? result + 1 : result;
-}
-
 const TPlanFragment* QuerySchedule::GetCoordFragment() const {
+  // Only have coordinator fragment for statements that return rows.
+  if (request_.stmt_type != TStmtType::QUERY) return nullptr;
   bool is_mt_exec = request_.query_ctx.request.query_options.mt_dop > 0;
   const TPlanFragment* fragment = is_mt_exec
       ? &request_.mt_plan_exec_info[0].fragments[0] : &request_.fragments[0];
-  if (fragment->partition.type == TPartitionType::UNPARTITIONED) {
+
     return fragment;
-  } else {
-    return nullptr;
-  }
 }
 
 void QuerySchedule::GetTPlanFragments(vector<const TPlanFragment*>* fragments) const {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/scheduling/query-schedule.h
----------------------------------------------------------------------
diff --git a/be/src/scheduling/query-schedule.h b/be/src/scheduling/query-schedule.h
index 39ce268..77c9cd6 100644
--- a/be/src/scheduling/query-schedule.h
+++ b/be/src/scheduling/query-schedule.h
@@ -140,34 +140,9 @@ class QuerySchedule {
   /// Helper methods used by scheduler to populate this QuerySchedule.
   void IncNumScanRanges(int64_t delta) { num_scan_ranges_ += delta; }
 
-  /// The following 4 functions need to be replaced once we stop special-casing
-  /// the coordinator instance in the coordinator.
-  /// The replacement is a single function int GetNumFInstances() (which includes
-  /// the coordinator instance).
-
-  /// TODO-MT: remove; this is actually only the number of remote instances
-  /// (from the coordinator's perspective)
-  void set_num_fragment_instances(int64_t num_fragment_instances) {
-    num_fragment_instances_ = num_fragment_instances;
-  }
-
-  /// Returns the number of fragment instances registered with this schedule.
-  /// MT: total number of fragment instances
-  /// ST: value set with set_num_fragment_instances(); excludes coord instance
-  /// (in effect the number of remote instances)
-  /// TODO-MT: get rid of special-casing of coordinator instance and always return the
-  /// total
+  /// Returns the total number of fragment instances.
   int GetNumFragmentInstances() const;
 
-  /// Returns the total number of fragment instances, incl. coordinator fragment.
-  /// TODO-MT: remove
-  int GetTotalFInstances() const;
-
-  /// Returns the number of remote fragment instances (excludes coordinator).
-  /// Works for both MT and ST.
-  /// TODO-MT: remove
-  int GetNumRemoteFInstances() const;
-
   /// Return the coordinator fragment, or nullptr if there isn't one.
   const TPlanFragment* GetCoordFragment() const;
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/scheduling/simple-scheduler.cc
----------------------------------------------------------------------
diff --git a/be/src/scheduling/simple-scheduler.cc b/be/src/scheduling/simple-scheduler.cc
index 5b6303e..9b52d5a 100644
--- a/be/src/scheduling/simple-scheduler.cc
+++ b/be/src/scheduling/simple-scheduler.cc
@@ -663,11 +663,6 @@ void SimpleScheduler::ComputeFragmentExecParams(const TQueryExecRequest& exec_re
           CreateInstanceId(schedule->query_id(), num_fragment_instances));
     }
   }
-  if (exec_request.fragments[0].partition.type == TPartitionType::UNPARTITIONED) {
-    // the root fragment is executed directly by the coordinator
-    --num_fragment_instances;
-  }
-  schedule->set_num_fragment_instances(num_fragment_instances);
 
   // compute destinations and # senders per exchange node
   // (the root fragment doesn't have a destination)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/fragment-exec-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fragment-exec-state.cc b/be/src/service/fragment-exec-state.cc
index 76e11d1..cc56c19 100644
--- a/be/src/service/fragment-exec-state.cc
+++ b/be/src/service/fragment-exec-state.cc
@@ -54,8 +54,10 @@ Status FragmentMgr::FragmentExecState::Prepare() {
 }
 
 void FragmentMgr::FragmentExecState::Exec() {
-  // Open() does the full execution, because all plan fragments have sinks
-  executor_.Open();
+  if (Prepare().ok()) {
+    executor_.Open();
+    executor_.Exec();
+  }
   executor_.Close();
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/fragment-exec-state.h
----------------------------------------------------------------------
diff --git a/be/src/service/fragment-exec-state.h b/be/src/service/fragment-exec-state.h
index 6cff7ce..c795cd8 100644
--- a/be/src/service/fragment-exec-state.h
+++ b/be/src/service/fragment-exec-state.h
@@ -47,9 +47,6 @@ class FragmentMgr::FragmentExecState {
   /// the fragment and returns OK.
   Status Cancel();
 
-  /// Call Prepare() and create and initialize data sink.
-  Status Prepare();
-
   /// Main loop of plan fragment execution. Blocks until execution finishes.
   void Exec();
 
@@ -67,6 +64,8 @@ class FragmentMgr::FragmentExecState {
   /// Publishes filter with ID 'filter_id' to this fragment's filter bank.
   void PublishFilter(int32_t filter_id, const TBloomFilter& thrift_bloom_filter);
 
+  PlanFragmentExecutor* executor() { return &executor_; }
+
  private:
   TQueryCtx query_ctx_;
   TPlanFragmentInstanceCtx fragment_instance_ctx_;
@@ -98,6 +97,9 @@ class FragmentMgr::FragmentExecState {
   /// the reporting RPC. `profile` may be NULL if a runtime profile has not been created
   /// for this fragment (e.g. when the fragment has failed during preparation).
   void ReportStatusCb(const Status& status, RuntimeProfile* profile, bool done);
+
+  /// Call Prepare() and create and initialize data sink.
+  Status Prepare();
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/fragment-mgr.cc
----------------------------------------------------------------------
diff --git a/be/src/service/fragment-mgr.cc b/be/src/service/fragment-mgr.cc
index 64e9a78..8e8fc05 100644
--- a/be/src/service/fragment-mgr.cc
+++ b/be/src/service/fragment-mgr.cc
@@ -54,9 +54,6 @@ Status FragmentMgr::ExecPlanFragment(const TExecPlanFragmentParams& exec_params)
     return process_mem_tracker->MemLimitExceeded(NULL, msg, 0);
   }
 
-  // Remote fragments must always have a sink. Remove when IMPALA-2905 is resolved.
-  DCHECK(exec_params.fragment_ctx.fragment.__isset.output_sink);
-
   shared_ptr<FragmentExecState> exec_state(
       new FragmentExecState(exec_params, ExecEnv::GetInstance()));
 
@@ -64,6 +61,8 @@ Status FragmentMgr::ExecPlanFragment(const TExecPlanFragmentParams& exec_params)
   // only happen after this RPC returns) can always find this fragment.
   {
     lock_guard<SpinLock> l(fragment_exec_state_map_lock_);
+    DCHECK(fragment_exec_state_map_.find(exec_state->fragment_instance_id())
+        == fragment_exec_state_map_.end());
     fragment_exec_state_map_.insert(
         make_pair(exec_state->fragment_instance_id(), exec_state));
   }
@@ -84,8 +83,7 @@ Status FragmentMgr::ExecPlanFragment(const TExecPlanFragmentParams& exec_params)
 void FragmentMgr::FragmentThread(TUniqueId fragment_instance_id) {
   shared_ptr<FragmentExecState> exec_state = GetFragmentExecState(fragment_instance_id);
   if (exec_state.get() == NULL) return;
-  Status status = exec_state->Prepare();
-  if (status.ok()) exec_state->Exec();
+  exec_state->Exec();
 
   // We're done with this plan fragment
   {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/impala-beeswax-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-beeswax-server.cc b/be/src/service/impala-beeswax-server.cc
index 3daa36b..ee7f958 100644
--- a/be/src/service/impala-beeswax-server.cc
+++ b/be/src/service/impala-beeswax-server.cc
@@ -17,55 +17,19 @@
 
 #include "service/impala-server.h"
 
-#include <algorithm>
 #include <boost/algorithm/string/join.hpp>
-#include <boost/date_time/posix_time/posix_time_types.hpp>
-#include <boost/unordered_set.hpp>
-#include <jni.h>
-#include <thrift/protocol/TDebugProtocol.h>
-#include <gtest/gtest.h>
-#include <boost/bind.hpp>
-#include <boost/algorithm/string.hpp>
-#include <gperftools/heap-profiler.h>
-#include <gperftools/malloc_extension.h>
-
-#include "codegen/llvm-codegen.h"
+
 #include "common/logging.h"
-#include "common/version.h"
-#include "exec/exec-node.h"
-#include "exec/hdfs-table-sink.h"
-#include "exec/scan-node.h"
-#include "exprs/expr.h"
-#include "runtime/data-stream-mgr.h"
-#include "runtime/client-cache.h"
-#include "runtime/descriptors.h"
-#include "runtime/data-stream-sender.h"
-#include "runtime/row-batch.h"
-#include "runtime/plan-fragment-executor.h"
-#include "runtime/hdfs-fs-cache.h"
+#include "gen-cpp/Frontend_types.h"
+#include "rpc/thrift-util.h"
 #include "runtime/exec-env.h"
-#include "runtime/mem-tracker.h"
 #include "runtime/raw-value.inline.h"
 #include "runtime/timestamp-value.h"
-#include "scheduling/simple-scheduler.h"
 #include "service/query-exec-state.h"
 #include "service/query-options.h"
-#include "util/container-util.h"
-#include "util/debug-util.h"
+#include "service/query-result-set.h"
 #include "util/impalad-metrics.h"
-#include "util/string-parser.h"
-#include "rpc/thrift-util.h"
-#include "rpc/thrift-server.h"
-#include "util/jni-util.h"
 #include "util/webserver.h"
-#include "gen-cpp/Types_types.h"
-#include "gen-cpp/ImpalaService.h"
-#include "gen-cpp/DataSinks_types.h"
-#include "gen-cpp/Types_types.h"
-#include "gen-cpp/ImpalaService.h"
-#include "gen-cpp/ImpalaService_types.h"
-#include "gen-cpp/ImpalaInternalService.h"
-#include "gen-cpp/Frontend_types.h"
 
 #include "common/names.h"
 
@@ -83,11 +47,17 @@ using namespace beeswax;
     }                                                           \
   } while (false)
 
+namespace {
+
+/// Ascii output precision for double/float
+constexpr int ASCII_PRECISION = 16;
+}
+
 namespace impala {
 
 // Ascii result set for Beeswax.
 // Beeswax returns rows in ascii, using "\t" as column delimiter.
-class ImpalaServer::AsciiQueryResultSet : public ImpalaServer::QueryResultSet {
+class AsciiQueryResultSet : public QueryResultSet {
  public:
   // Rows are added into rowset.
   AsciiQueryResultSet(const TResultSetMetadata& metadata, vector<string>* rowset)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/impala-hs2-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-hs2-server.cc b/be/src/service/impala-hs2-server.cc
index ee79b4b..de0e2f3 100644
--- a/be/src/service/impala-hs2-server.cc
+++ b/be/src/service/impala-hs2-server.cc
@@ -36,13 +36,14 @@
 #include "exprs/expr.h"
 #include "rpc/thrift-util.h"
 #include "runtime/raw-value.h"
+#include "service/hs2-util.h"
 #include "service/query-exec-state.h"
 #include "service/query-options.h"
+#include "service/query-result-set.h"
 #include "util/debug-util.h"
-#include "util/runtime-profile-counters.h"
 #include "util/impalad-metrics.h"
+#include "util/runtime-profile-counters.h"
 #include "util/string-parser.h"
-#include "service/hs2-util.h"
 
 #include "common/names.h"
 
@@ -129,7 +130,7 @@ static TOperationState::type QueryStateToTOperationState(
 
 // Result set container for Hive protocol versions >= V6, where results are returned in
 // column-orientation.
-class ImpalaServer::HS2ColumnarResultSet : public ImpalaServer::QueryResultSet {
+class HS2ColumnarResultSet : public QueryResultSet {
  public:
   HS2ColumnarResultSet(const TResultSetMetadata& metadata, TRowSet* rowset = NULL)
       : metadata_(metadata), result_set_(rowset), num_rows_(0) {
@@ -317,7 +318,7 @@ class ImpalaServer::HS2ColumnarResultSet : public ImpalaServer::QueryResultSet {
 };
 
 // TRow result set for HiveServer2
-class ImpalaServer::HS2RowOrientedResultSet : public ImpalaServer::QueryResultSet {
+class HS2RowOrientedResultSet : public QueryResultSet {
  public:
   // Rows are added into rowset.
   HS2RowOrientedResultSet(const TResultSetMetadata& metadata, TRowSet* rowset = NULL)
@@ -393,16 +394,6 @@ class ImpalaServer::HS2RowOrientedResultSet : public ImpalaServer::QueryResultSe
   scoped_ptr<TRowSet> owned_result_set_;
 };
 
-ImpalaServer::QueryResultSet* ImpalaServer::CreateHS2ResultSet(
-    TProtocolVersion::type version, const TResultSetMetadata& metadata,
-    TRowSet* rowset) {
-  if (version < TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V6) {
-    return new HS2RowOrientedResultSet(metadata, rowset);
-  } else {
-    return new HS2ColumnarResultSet(metadata, rowset);
-  }
-}
-
 void ImpalaServer::ExecuteMetadataOp(const THandleIdentifier& session_handle,
     TMetadataOpRequest* request, TOperationHandle* handle, thrift::TStatus* status) {
   TUniqueId session_id;
@@ -482,6 +473,18 @@ void ImpalaServer::ExecuteMetadataOp(const THandleIdentifier& session_handle,
   status->__set_statusCode(thrift::TStatusCode::SUCCESS_STATUS);
 }
 
+namespace {
+
+QueryResultSet* CreateHS2ResultSet(
+    TProtocolVersion::type version, const TResultSetMetadata& metadata, TRowSet* rowset) {
+  if (version < TProtocolVersion::HIVE_CLI_SERVICE_PROTOCOL_V6) {
+    return new HS2RowOrientedResultSet(metadata, rowset);
+  } else {
+    return new HS2ColumnarResultSet(metadata, rowset);
+  }
+}
+}
+
 Status ImpalaServer::FetchInternal(const TUniqueId& query_id, int32_t fetch_size,
     bool fetch_first, TFetchResultsResp* fetch_results) {
   shared_ptr<QueryExecState> exec_state = GetQueryExecState(query_id, false);
@@ -759,8 +762,9 @@ void ImpalaServer::ExecuteStatement(TExecuteStatementResp& return_val,
 
   // Optionally enable result caching on the QueryExecState.
   if (cache_num_rows > 0) {
-    status = exec_state->SetResultCache(CreateHS2ResultSet(session->hs2_version,
-            *exec_state->result_metadata()), cache_num_rows);
+    status = exec_state->SetResultCache(
+        CreateHS2ResultSet(session->hs2_version, *exec_state->result_metadata(), nullptr),
+        cache_num_rows);
     if (!status.ok()) {
       UnregisterQuery(exec_state->query_id(), false, &status);
       HS2_RETURN_ERROR(return_val, status.GetDetail(), SQLSTATE_GENERAL_ERROR);

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/impala-internal-service.h
----------------------------------------------------------------------
diff --git a/be/src/service/impala-internal-service.h b/be/src/service/impala-internal-service.h
index a238f65..af54c35 100644
--- a/be/src/service/impala-internal-service.h
+++ b/be/src/service/impala-internal-service.h
@@ -18,8 +18,6 @@
 #ifndef IMPALA_SERVICE_IMPALA_INTERNAL_SERVICE_H
 #define IMPALA_SERVICE_IMPALA_INTERNAL_SERVICE_H
 
-#include <boost/shared_ptr.hpp>
-
 #include "gen-cpp/ImpalaInternalService.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
 #include "service/impala-server.h"
@@ -32,9 +30,12 @@ namespace impala {
 /// ImpalaInternalService service.
 class ImpalaInternalService : public ImpalaInternalServiceIf {
  public:
-  ImpalaInternalService(const boost::shared_ptr<ImpalaServer>& impala_server,
-      const boost::shared_ptr<FragmentMgr>& fragment_mgr)
-      : impala_server_(impala_server), fragment_mgr_(fragment_mgr) { }
+  ImpalaInternalService() {
+    impala_server_ = ExecEnv::GetInstance()->impala_server();
+    DCHECK(impala_server_ != nullptr);
+    fragment_mgr_ = ExecEnv::GetInstance()->fragment_mgr();
+    DCHECK(fragment_mgr_ != nullptr);
+  }
 
   virtual void ExecPlanFragment(TExecPlanFragmentResult& return_val,
       const TExecPlanFragmentParams& params) {
@@ -74,10 +75,10 @@ class ImpalaInternalService : public ImpalaInternalServiceIf {
 
  private:
   /// Manages fragment reporting and data transmission
-  boost::shared_ptr<ImpalaServer> impala_server_;
+  ImpalaServer* impala_server_;
 
   /// Manages fragment execution
-  boost::shared_ptr<FragmentMgr> fragment_mgr_;
+  FragmentMgr* fragment_mgr_;
 };
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/impala-server.cc
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.cc b/be/src/service/impala-server.cc
index 7f9d862..bf83eec 100644
--- a/be/src/service/impala-server.cc
+++ b/be/src/service/impala-server.cc
@@ -206,7 +206,6 @@ const string HS2_SERVER_NAME = "hiveserver2-frontend";
 const char* ImpalaServer::SQLSTATE_SYNTAX_ERROR_OR_ACCESS_VIOLATION = "42000";
 const char* ImpalaServer::SQLSTATE_GENERAL_ERROR = "HY000";
 const char* ImpalaServer::SQLSTATE_OPTIONAL_FEATURE_NOT_IMPLEMENTED = "HYC00";
-const int ImpalaServer::ASCII_PRECISION = 16; // print 16 digits for double/float
 
 const int MAX_NM_MISSED_HEARTBEATS = 5;
 
@@ -1866,9 +1865,7 @@ Status CreateImpalaServer(ExecEnv* exec_env, int beeswax_port, int hs2_port, int
   }
 
   if (be_port != 0 && be_server != NULL) {
-    boost::shared_ptr<FragmentMgr> fragment_mgr(new FragmentMgr());
-    boost::shared_ptr<ImpalaInternalService> thrift_if(
-        new ImpalaInternalService(handler, fragment_mgr));
+    boost::shared_ptr<ImpalaInternalService> thrift_if(new ImpalaInternalService());
     boost::shared_ptr<TProcessor> be_processor(
         new ImpalaInternalServiceProcessor(thrift_if));
     boost::shared_ptr<TProcessorEventHandler> event_handler(

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/impala-server.h
----------------------------------------------------------------------
diff --git a/be/src/service/impala-server.h b/be/src/service/impala-server.h
index 2104c5e..53f3384 100644
--- a/be/src/service/impala-server.h
+++ b/be/src/service/impala-server.h
@@ -249,45 +249,6 @@ class ImpalaServer : public ImpalaServiceIf, public ImpalaHiveServer2ServiceIf,
 
   boost::scoped_ptr<ImpalaHttpHandler> http_handler_;
 
-  /// Query result set stores converted rows returned by QueryExecState.fetchRows(). It
-  /// provides an interface to convert Impala rows to external API rows.
-  /// It is an abstract class. Subclass must implement AddOneRow().
-  class QueryResultSet {
-   public:
-    QueryResultSet() {}
-    virtual ~QueryResultSet() {}
-
-    /// Add the row (list of expr value) from a select query to this result set. When a row
-    /// comes from a select query, the row is in the form of expr values (void*). 'scales'
-    /// contains the values' scales (# of digits after decimal), with -1 indicating no
-    /// scale specified.
-    virtual Status AddOneRow(
-        const std::vector<void*>& row, const std::vector<int>& scales) = 0;
-
-    /// Add the TResultRow to this result set. When a row comes from a DDL/metadata
-    /// operation, the row in the form of TResultRow.
-    virtual Status AddOneRow(const TResultRow& row) = 0;
-
-    /// Copies rows in the range [start_idx, start_idx + num_rows) from the other result
-    /// set into this result set. Returns the number of rows added to this result set.
-    /// Returns 0 if the given range is out of bounds of the other result set.
-    virtual int AddRows(const QueryResultSet* other, int start_idx, int num_rows) = 0;
-
-    /// Returns the approximate size of this result set in bytes.
-    int64_t ByteSize() { return ByteSize(0, size()); }
-
-    /// Returns the approximate size of the given range of rows in bytes.
-    virtual int64_t ByteSize(int start_idx, int num_rows) = 0;
-
-    /// Returns the size of this result set in number of rows.
-    virtual size_t size() = 0;
-  };
-
-  /// Result set implementations for Beeswax and HS2
-  class AsciiQueryResultSet;
-  class HS2RowOrientedResultSet;
-  class HS2ColumnarResultSet;
-
   struct SessionState;
 
   /// Execution state of a query.
@@ -299,14 +260,6 @@ class ImpalaServer : public ImpalaServiceIf, public ImpalaHiveServer2ServiceIf,
   static const char* SQLSTATE_GENERAL_ERROR;
   static const char* SQLSTATE_OPTIONAL_FEATURE_NOT_IMPLEMENTED;
 
-  /// Ascii output precision for double/float
-  static const int ASCII_PRECISION;
-
-  QueryResultSet* CreateHS2ResultSet(
-      apache::hive::service::cli::thrift::TProtocolVersion::type version,
-      const TResultSetMetadata& metadata,
-      apache::hive::service::cli::thrift::TRowSet* rowset = NULL);
-
   /// Return exec state for given query_id, or NULL if not found.
   /// If 'lock' is true, the returned exec state's lock() will be acquired before
   /// the query_exec_state_map_lock_ is released.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/query-exec-state.cc
----------------------------------------------------------------------
diff --git a/be/src/service/query-exec-state.cc b/be/src/service/query-exec-state.cc
index d55ac54..1532ecf 100644
--- a/be/src/service/query-exec-state.cc
+++ b/be/src/service/query-exec-state.cc
@@ -19,14 +19,15 @@
 #include <limits>
 #include <gutil/strings/substitute.h>
 
-#include "exprs/expr.h"
 #include "exprs/expr-context.h"
+#include "exprs/expr.h"
 #include "runtime/mem-tracker.h"
 #include "runtime/row-batch.h"
 #include "runtime/runtime-state.h"
-#include "service/impala-server.h"
 #include "service/frontend.h"
+#include "service/impala-server.h"
 #include "service/query-options.h"
+#include "service/query-result-set.h"
 #include "util/debug-util.h"
 #include "util/impalad-metrics.h"
 #include "util/runtime-profile-counters.h"
@@ -191,6 +192,7 @@ Status ImpalaServer::QueryExecState::Exec(TExecRequest* exec_request) {
             exec_request_.set_query_option_request.value,
             &session_->default_query_options,
             &session_->set_query_options_mask));
+        SetResultSet({}, {});
       } else {
         // "SET" returns a table of all query options.
         map<string, string> config;
@@ -421,17 +423,10 @@ Status ImpalaServer::QueryExecState::ExecQueryOrDmlRequest(
     summary_profile_.AddInfoString(TABLES_WITH_CORRUPT_STATS_KEY, ss.str());
   }
 
-  // If desc_tbl is not set, query has SELECT with no FROM. In that
-  // case, the query can only have a single fragment, and that fragment needs to be
-  // executed by the coordinator. This check confirms that.
-  // If desc_tbl is set, the query may or may not have a coordinator fragment.
   bool is_mt_exec = query_exec_request.query_ctx.request.query_options.mt_dop > 0;
   const TPlanFragment& fragment = is_mt_exec
       ? query_exec_request.mt_plan_exec_info[0].fragments[0]
       : query_exec_request.fragments[0];
-  bool has_coordinator_fragment =
-      fragment.partition.type == TPartitionType::UNPARTITIONED;
-  DCHECK(has_coordinator_fragment || query_exec_request.__isset.desc_tbl);
 
   {
     lock_guard<mutex> l(lock_);
@@ -449,7 +444,7 @@ Status ImpalaServer::QueryExecState::ExecQueryOrDmlRequest(
   }
 
   coord_.reset(new Coordinator(*schedule_, exec_env_, query_events_));
-  status = coord_->Exec(&output_expr_ctxs_);
+  status = coord_->Exec();
   {
     lock_guard<mutex> l(lock_);
     RETURN_IF_ERROR(UpdateQueryStatus(status));
@@ -538,12 +533,11 @@ void ImpalaServer::QueryExecState::Done() {
   query_events_->MarkEvent("Unregister query");
 
   if (coord_.get() != NULL) {
-    Expr::Close(output_expr_ctxs_, coord_->runtime_state());
     // Release any reserved resources.
     Status status = exec_env_->scheduler()->Release(schedule_.get());
     if (!status.ok()) {
       LOG(WARNING) << "Failed to release resources of query " << schedule_->query_id()
-            << " because of error: " << status.GetDetail();
+                   << " because of error: " << status.GetDetail();
     }
     coord_->TearDown();
   }
@@ -626,7 +620,6 @@ Status ImpalaServer::QueryExecState::WaitInternal() {
 
   if (coord_.get() != NULL) {
     RETURN_IF_ERROR(coord_->Wait());
-    RETURN_IF_ERROR(Expr::Open(output_expr_ctxs_, coord_->runtime_state()));
     RETURN_IF_ERROR(UpdateCatalog());
   }
 
@@ -719,6 +712,10 @@ Status ImpalaServer::QueryExecState::FetchRowsInternal(const int32_t max_rows,
     return Status::OK();
   }
 
+  if (coord_.get() == nullptr) {
+    return Status("Client tried to fetch rows on a query that produces no results.");
+  }
+
   int32_t num_rows_fetched_from_cache = 0;
   if (result_cache_max_size_ > 0 && result_cache_ != NULL) {
     // Satisfy the fetch from the result cache if possible.
@@ -729,27 +726,7 @@ Status ImpalaServer::QueryExecState::FetchRowsInternal(const int32_t max_rows,
     if (num_rows_fetched_from_cache >= max_rows) return Status::OK();
   }
 
-  // List of expr values to hold evaluated rows from the query
-  vector<void*> result_row;
-  result_row.resize(output_expr_ctxs_.size());
-
-  // List of scales for floating point values in result_row
-  vector<int> scales;
-  scales.resize(result_row.size());
-
-  if (coord_ == NULL) {
-    // Query with LIMIT 0.
-    query_state_ = QueryState::FINISHED;
-    eos_ = true;
-    return Status::OK();
-  }
-
   query_state_ = QueryState::FINISHED;  // results will be ready after this call
-  // Fetch the next batch if we've returned the current batch entirely
-  if (current_batch_ == NULL || current_batch_row_ >= current_batch_->num_rows()) {
-    RETURN_IF_ERROR(FetchNextBatch());
-  }
-  if (current_batch_ == NULL) return Status::OK();
 
   // Maximum number of rows to be fetched from the coord.
   int32_t max_coord_rows = max_rows;
@@ -759,22 +736,26 @@ Status ImpalaServer::QueryExecState::FetchRowsInternal(const int32_t max_rows,
   }
   {
     SCOPED_TIMER(row_materialization_timer_);
-    // Convert the available rows, limited by max_coord_rows
-    int available = current_batch_->num_rows() - current_batch_row_;
-    int fetched_count = available;
-    // max_coord_rows <= 0 means no limit
-    if (max_coord_rows > 0 && max_coord_rows < available) fetched_count = max_coord_rows;
-    for (int i = 0; i < fetched_count; ++i) {
-      TupleRow* row = current_batch_->GetRow(current_batch_row_);
-      RETURN_IF_ERROR(GetRowValue(row, &result_row, &scales));
-      RETURN_IF_ERROR(fetched_rows->AddOneRow(result_row, scales));
-      ++num_rows_fetched_;
-      ++current_batch_row_;
+    size_t before = fetched_rows->size();
+    // Temporarily release lock so calls to Cancel() are not blocked. fetch_rows_lock_
+    // (already held) ensures that we do not call coord_->GetNext() multiple times
+    // concurrently.
+    // TODO: Simplify this.
+    lock_.unlock();
+    Status status = coord_->GetNext(fetched_rows, max_coord_rows, &eos_);
+    lock_.lock();
+    int num_fetched = fetched_rows->size() - before;
+    DCHECK(max_coord_rows <= 0 || num_fetched <= max_coord_rows) << Substitute(
+        "Fetched more rows ($0) than asked for ($1)", num_fetched, max_coord_rows);
+    num_rows_fetched_ += num_fetched;
+
+    RETURN_IF_ERROR(status);
+    // Check if query status has changed during GetNext() call
+    if (!query_status_.ok()) {
+      eos_ = true;
+      return query_status_;
     }
   }
-  ExprContext::FreeLocalAllocations(output_expr_ctxs_);
-  // Check if there was an error evaluating a row value.
-  RETURN_IF_ERROR(coord_->runtime_state()->CheckQueryState());
 
   // Update the result cache if necessary.
   if (result_cache_max_size_ > 0 && result_cache_.get() != NULL) {
@@ -833,16 +814,6 @@ Status ImpalaServer::QueryExecState::FetchRowsInternal(const int32_t max_rows,
   return Status::OK();
 }
 
-Status ImpalaServer::QueryExecState::GetRowValue(TupleRow* row, vector<void*>* result,
-                                                 vector<int>* scales) {
-  DCHECK(result->size() >= output_expr_ctxs_.size());
-  for (int i = 0; i < output_expr_ctxs_.size(); ++i) {
-    (*result)[i] = output_expr_ctxs_[i]->GetValue(row);
-    (*scales)[i] = output_expr_ctxs_[i]->root()->output_scale();
-  }
-  return Status::OK();
-}
-
 Status ImpalaServer::QueryExecState::Cancel(bool check_inflight, const Status* cause) {
   Coordinator* coord;
   {
@@ -931,28 +902,6 @@ Status ImpalaServer::QueryExecState::UpdateCatalog() {
   return Status::OK();
 }
 
-Status ImpalaServer::QueryExecState::FetchNextBatch() {
-  DCHECK(!eos_);
-  DCHECK(coord_.get() != NULL);
-
-  // Temporarily release lock so calls to Cancel() are not blocked.  fetch_rows_lock_
-  // ensures that we do not call coord_->GetNext() multiple times concurrently.
-  lock_.unlock();
-  Status status = coord_->GetNext(&current_batch_, coord_->runtime_state());
-  lock_.lock();
-  if (!status.ok()) return status;
-
-  // Check if query status has changed during GetNext() call
-  if (!query_status_.ok()) {
-    current_batch_ = NULL;
-    return query_status_;
-  }
-
-  current_batch_row_ = 0;
-  eos_ = current_batch_ == NULL;
-  return Status::OK();
-}
-
 void ImpalaServer::QueryExecState::SetResultSet(const vector<string>& results) {
   request_result_set_.reset(new vector<TResultRow>);
   request_result_set_->resize(results.size());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/query-exec-state.h
----------------------------------------------------------------------
diff --git a/be/src/service/query-exec-state.h b/be/src/service/query-exec-state.h
index 0a763ff..54ee929 100644
--- a/be/src/service/query-exec-state.h
+++ b/be/src/service/query-exec-state.h
@@ -248,7 +248,7 @@ class ImpalaServer::QueryExecState {
   /// Resource assignment determined by scheduler. Owned by obj_pool_.
   boost::scoped_ptr<QuerySchedule> schedule_;
 
-  /// not set for ddl queries, or queries with "limit 0"
+  /// Not set for ddl queries.
   boost::scoped_ptr<Coordinator> coord_;
 
   /// Runs statements that query or modify the catalog via the CatalogService.
@@ -293,7 +293,7 @@ class ImpalaServer::QueryExecState {
   MonotonicStopWatch client_wait_sw_;
 
   RuntimeProfile::EventSequence* query_events_;
-  std::vector<ExprContext*> output_expr_ctxs_;
+
   bool is_cancelled_; // if true, Cancel() was called.
   bool eos_;  // if true, there are no more rows to return
   // We enforce the invariant that query_status_ is not OK iff query_state_
@@ -356,13 +356,6 @@ class ImpalaServer::QueryExecState {
   /// Caller needs to hold fetch_rows_lock_ and lock_.
   Status FetchRowsInternal(const int32_t max_rows, QueryResultSet* fetched_rows);
 
-  /// Fetch the next row batch and store the results in current_batch_. Only called for
-  /// non-DDL / DML queries. current_batch_ is set to NULL if execution is complete or the
-  /// query was cancelled.
-  /// Caller needs to hold fetch_rows_lock_ and lock_. Blocks, during which time lock_ is
-  /// released.
-  Status FetchNextBatch();
-
   /// Evaluates 'output_expr_ctxs_' against 'row' and output the evaluated row in
   /// 'result'. The values' scales (# of digits after decimal) are stored in 'scales'.
   /// result and scales must have been resized to the number of columns before call.

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/service/query-result-set.h
----------------------------------------------------------------------
diff --git a/be/src/service/query-result-set.h b/be/src/service/query-result-set.h
new file mode 100644
index 0000000..b444ca3
--- /dev/null
+++ b/be/src/service/query-result-set.h
@@ -0,0 +1,64 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef IMPALA_SERVICE_QUERY_RESULT_SET_H
+#define IMPALA_SERVICE_QUERY_RESULT_SET_H
+
+#include "common/status.h"
+#include "gen-cpp/Data_types.h"
+
+#include <vector>
+
+namespace impala {
+
+/// Stores client-ready query result rows returned by
+/// QueryExecState::FetchRows(). Subclasses implement AddRows() / AddOneRow() to
+/// specialise how Impala's row batches are converted to client-API result
+/// representations.
+class QueryResultSet {
+ public:
+  QueryResultSet() {}
+  virtual ~QueryResultSet() {}
+
+  /// Add a single row to this result set. The row is a vector of pointers to values,
+  /// whose memory belongs to the caller. 'scales' contains the scales for decimal values
+  /// (# of digits after decimal), with -1 indicating no scale specified or the
+  /// corresponding value is not a decimal.
+  virtual Status AddOneRow(
+      const std::vector<void*>& row, const std::vector<int>& scales) = 0;
+
+  /// Add the TResultRow to this result set. When a row comes from a DDL/metadata
+  /// operation, the row in the form of TResultRow.
+  virtual Status AddOneRow(const TResultRow& row) = 0;
+
+  /// Copies rows in the range [start_idx, start_idx + num_rows) from the other result
+  /// set into this result set. Returns the number of rows added to this result set.
+  /// Returns 0 if the given range is out of bounds of the other result set.
+  virtual int AddRows(const QueryResultSet* other, int start_idx, int num_rows) = 0;
+
+  /// Returns the approximate size of this result set in bytes.
+  int64_t ByteSize() { return ByteSize(0, size()); }
+
+  /// Returns the approximate size of the given range of rows in bytes.
+  virtual int64_t ByteSize(int start_idx, int num_rows) = 0;
+
+  /// Returns the size of this result set in number of rows.
+  virtual size_t size() = 0;
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/testutil/in-process-servers.cc
----------------------------------------------------------------------
diff --git a/be/src/testutil/in-process-servers.cc b/be/src/testutil/in-process-servers.cc
index cf0e28e..b28f7fc 100644
--- a/be/src/testutil/in-process-servers.cc
+++ b/be/src/testutil/in-process-servers.cc
@@ -34,6 +34,7 @@
 
 DECLARE_string(ssl_server_certificate);
 DECLARE_string(ssl_private_key);
+DECLARE_int32(be_port);
 
 using namespace apache::thrift;
 using namespace impala;
@@ -43,6 +44,9 @@ InProcessImpalaServer* InProcessImpalaServer::StartWithEphemeralPorts(
   for (int tries = 0; tries < 10; ++tries) {
     int backend_port = FindUnusedEphemeralPort();
     if (backend_port == -1) continue;
+    // This flag is read directly in several places to find the address of the local
+    // backend interface.
+    FLAGS_be_port = backend_port;
 
     int subscriber_port = FindUnusedEphemeralPort();
     if (subscriber_port == -1) continue;

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/common/thrift/DataSinks.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/DataSinks.thrift b/common/thrift/DataSinks.thrift
index 12a75b9..83c63b7 100644
--- a/common/thrift/DataSinks.thrift
+++ b/common/thrift/DataSinks.thrift
@@ -26,7 +26,8 @@ include "Partitions.thrift"
 enum TDataSinkType {
   DATA_STREAM_SINK,
   TABLE_SINK,
-  JOIN_BUILD_SINK
+  JOIN_BUILD_SINK,
+  PLAN_ROOT_SINK
 }
 
 enum TSinkAction {
@@ -87,10 +88,10 @@ struct TJoinBuildSink {
 
 // Union type of all table sinks.
 struct TTableSink {
-  1: required Types.TTableId  target_table_id
+  1: required Types.TTableId target_table_id
   2: required TTableSinkType type
   3: required TSinkAction action
-  4: optional THdfsTableSink  hdfs_table_sink
+  4: optional THdfsTableSink hdfs_table_sink
   5: optional TKuduTableSink kudu_table_sink
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java b/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
index b02bc73..392b961 100644
--- a/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/QueryStmt.java
@@ -25,6 +25,8 @@ import java.util.Set;
 import org.apache.impala.catalog.Type;
 import org.apache.impala.common.AnalysisException;
 import org.apache.impala.common.TreeNode;
+import org.apache.impala.planner.DataSink;
+import org.apache.impala.planner.PlanRootSink;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Lists;
@@ -409,6 +411,10 @@ public abstract class QueryStmt extends StatementBase {
     resultExprs_ = Expr.substituteList(resultExprs_, smap, analyzer, true);
   }
 
+  public DataSink createDataSink() {
+    return new PlanRootSink();
+  }
+
   public ArrayList<OrderByElement> cloneOrderByElements() {
     if (orderByElements_ == null) return null;
     ArrayList<OrderByElement> result =

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
new file mode 100644
index 0000000..a199f54
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/planner/PlanRootSink.java
@@ -0,0 +1,39 @@
+// 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.
+
+package org.apache.impala.planner;
+
+import org.apache.impala.thrift.TDataSink;
+import org.apache.impala.thrift.TDataSinkType;
+import org.apache.impala.thrift.TExplainLevel;
+
+/**
+ * Sink for the root of a query plan that produces result rows. Allows coordination
+ * between the sender which produces those rows, and the consumer which sends them to the
+ * client, despite both executing concurrently.
+ */
+public class PlanRootSink extends DataSink {
+
+  public String getExplainString(String prefix, String detailPrefix,
+      TExplainLevel explainLevel) {
+    return String.format("%sPLAN-ROOT SINK\n", prefix);
+  }
+
+  protected TDataSink toThrift() {
+    return new TDataSink(TDataSinkType.PLAN_ROOT_SINK);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/fe/src/main/java/org/apache/impala/planner/Planner.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/Planner.java b/fe/src/main/java/org/apache/impala/planner/Planner.java
index 405eebe..ed4c677 100644
--- a/fe/src/main/java/org/apache/impala/planner/Planner.java
+++ b/fe/src/main/java/org/apache/impala/planner/Planner.java
@@ -149,6 +149,8 @@ public class Planner {
       } else if (ctx_.isDelete()) {
         // Set up delete sink for root fragment
         rootFragment.setSink(ctx_.getAnalysisResult().getDeleteStmt().createDataSink());
+      } else if (ctx_.isQuery()) {
+        rootFragment.setSink(ctx_.getAnalysisResult().getQueryStmt().createDataSink());
       }
       QueryStmt queryStmt = ctx_.getQueryStmt();
       queryStmt.substituteResultExprs(rootNodeSmap, ctx_.getRootAnalyzer());

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/fe/src/main/java/org/apache/impala/planner/PlannerContext.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/planner/PlannerContext.java b/fe/src/main/java/org/apache/impala/planner/PlannerContext.java
index 29cca13..3275a7a 100644
--- a/fe/src/main/java/org/apache/impala/planner/PlannerContext.java
+++ b/fe/src/main/java/org/apache/impala/planner/PlannerContext.java
@@ -90,6 +90,7 @@ public class PlannerContext {
   public boolean isInsertOrCtas() {
     return analysisResult_.isInsertStmt() || analysisResult_.isCreateTableAsSelectStmt();
   }
+  public boolean isQuery() { return analysisResult_.isQueryStmt(); }
 
   public boolean hasSubplan() { return !subplans_.isEmpty(); }
   public SubplanNode getSubplan() { return subplans_.getFirst(); }

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test b/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
index 47bfb23..d7838f9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/aggregation.test
@@ -3,12 +3,16 @@ select count(*), count(tinyint_col), min(tinyint_col), max(tinyint_col), sum(tin
 avg(tinyint_col)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*), count(tinyint_col), min(tinyint_col), max(tinyint_col), sum(tinyint_col), avg(tinyint_col)
 |
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count:merge(*), count:merge(tinyint_col), min:merge(tinyint_col), max:merge(tinyint_col), sum:merge(tinyint_col), avg:merge(tinyint_col)
 |
@@ -26,6 +30,8 @@ avg(tinyint_col)
 from functional.alltypesagg
 group by 2, 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*), min(tinyint_col), max(tinyint_col), sum(tinyint_col), avg(tinyint_col)
 |  group by: bigint_col, tinyint_col
@@ -33,6 +39,8 @@ group by 2, 1
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -54,6 +62,8 @@ from functional.testtbl
 having count(id) > 0
 order by avg(zip) limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=10]
 |  order by: avg(zip) ASC
 |
@@ -64,6 +74,8 @@ order by avg(zip) limit 10
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=10]
 |  order by: avg(zip) ASC
 |
@@ -85,6 +97,8 @@ from functional.alltypesagg
 group by int_col + int_col, int_col * int_col, int_col + int_col
 having (int_col * int_col) < 0 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: int_col + int_col, int_col * int_col
 |  having: int_col * int_col < 0
@@ -93,6 +107,8 @@ having (int_col * int_col) < 0 limit 10
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |  limit: 10
 |
@@ -116,6 +132,8 @@ functional.alltypes t1 inner join functional.alltypestiny t2
 group by t1.tinyint_col, t2.smallint_col
 having count(t2.int_col) = count(t1.bigint_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*), count(t2.int_col), count(t1.bigint_col)
 |  group by: t1.tinyint_col, t2.smallint_col
@@ -141,6 +159,8 @@ select 1 from
    group by int_col) t
 where t.x > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: avg(bigint_col)
 |  group by: int_col
@@ -157,6 +177,8 @@ select count(*) from
    select * from functional.alltypessmall) t
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  limit: 10
@@ -169,6 +191,8 @@ limit 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |  limit: 10
@@ -194,6 +218,8 @@ select count(*) from
 group by t.bigint_col
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: bigint_col
@@ -207,6 +233,8 @@ limit 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |  limit: 10
 |
@@ -237,6 +265,8 @@ from
    select * from functional.alltypessmall) t
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(int_col)
 |  limit: 10
@@ -252,6 +282,8 @@ limit 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:AGGREGATE [FINALIZE]
 |  output: count:merge(int_col)
 |  limit: 10
@@ -286,6 +318,8 @@ from
 group by t.bigint_col
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(int_col)
 |  group by: t.bigint_col
@@ -302,6 +336,8 @@ limit 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |  limit: 10
 |
@@ -334,6 +370,8 @@ from
    select * from functional.alltypessmall) t
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(int_col), count:merge(smallint_col)
 |  limit: 10
@@ -350,6 +388,8 @@ limit 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:AGGREGATE [FINALIZE]
 |  output: count:merge(int_col), count:merge(smallint_col)
 |  limit: 10
@@ -386,6 +426,8 @@ from
 group by t.bigint_col
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(int_col), count:merge(smallint_col)
 |  group by: t.bigint_col
@@ -403,6 +445,8 @@ limit 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |  limit: 10
 |
@@ -438,6 +482,8 @@ from
 group by t.bigint_col
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(int_col), count:merge(smallint_col)
 |  group by: t.bigint_col
@@ -458,6 +504,8 @@ limit 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:EXCHANGE [UNPARTITIONED]
 |  limit: 10
 |
@@ -495,6 +543,8 @@ limit 10
 # test that aggregations are not placed below an unpartitioned exchange with a limit
 select count(*) from (select * from functional.alltypes limit 10) t
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -502,6 +552,8 @@ select count(*) from (select * from functional.alltypes limit 10) t
    partitions=24/24 files=24 size=478.45KB
    limit: 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -518,6 +570,8 @@ select count(*) from
    union all
    (select * from functional.alltypessmall) limit 10) t
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -530,6 +584,8 @@ select count(*) from
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -555,6 +611,8 @@ select * from (
   limit 2) v
 limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count(cnt)
 |  limit: 1
@@ -580,6 +638,8 @@ limit 1
    partitions=11/11 files=11 size=814.73KB
    runtime filters: RF000 -> t1.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count(cnt)
 |  limit: 1
@@ -629,6 +689,8 @@ select * from
    group by 1, 2, 3, 4) v
 where v.a = v.b and v.b = v.c and v.c = v.d and v.a = v.c and v.a = v.d
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: tinyint_col, smallint_col, int_col + int_col, coalesce(bigint_col, year)
 |  having: int_col + int_col = coalesce(bigint_col, year), smallint_col = int_col + int_col
@@ -643,6 +705,8 @@ select cnt from
    from functional.alltypestiny
    group by bool_col, x) v
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: bool_col, CAST(NULL AS INT)
@@ -656,6 +720,8 @@ select cnt from
    from functional.alltypestiny
    group by bool_col, x) v
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(int_col)
 |  group by: bool_col, NULL
@@ -669,6 +735,8 @@ select cnt from
 # test simple group_concat with distinct
 select group_concat(distinct string_col) from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: group_concat(string_col)
 |
@@ -678,6 +746,8 @@ select group_concat(distinct string_col) from functional.alltypesagg
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: group_concat:merge(string_col)
 |
@@ -702,6 +772,8 @@ select day, group_concat(distinct string_col)
 from (select * from functional.alltypesagg where id % 100 = day order by id limit 99999) a
 group by day
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: group_concat(string_col)
 |  group by: day
@@ -716,6 +788,8 @@ group by day
    partitions=11/11 files=11 size=814.73KB
    predicates: id % 100 = day
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: group_concat(string_col)
 |  group by: day
@@ -739,6 +813,8 @@ select count(distinct cast(timestamp_col as string)),
 group_concat(distinct cast(timestamp_col as string))
 from functional.alltypesagg group by year
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(CAST(timestamp_col AS STRING)), group_concat(CAST(timestamp_col AS STRING))
 |  group by: year
@@ -749,6 +825,8 @@ from functional.alltypesagg group by year
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:AGGREGATE [FINALIZE]
@@ -769,6 +847,8 @@ from functional.alltypesagg group by year
 # test group_concat distinct with other non-distinct aggregate functions
  select group_concat(distinct string_col), count(*) from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: group_concat(string_col), count:merge(*)
 |
@@ -779,6 +859,8 @@ from functional.alltypesagg group by year
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: group_concat:merge(string_col), count:merge(*)
 |
@@ -804,6 +886,8 @@ from functional.alltypesagg group by year
 select group_concat(distinct string_col, '-'), sum(int_col), count(distinct string_col)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: group_concat(string_col, '-'), count(string_col), sum:merge(int_col)
 |
@@ -814,6 +898,8 @@ from functional.alltypesagg
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: group_concat:merge(string_col, '-'), count:merge(string_col), sum:merge(int_col)
 |
@@ -841,6 +927,8 @@ select month, year, count(*), count(distinct date_string_col),
 group_concat(distinct date_string_col, '-') from functional.alltypesagg
 group by month, year
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(date_string_col), group_concat(date_string_col, '-'), count:merge(*)
 |  group by: month, year
@@ -852,6 +940,8 @@ group by month, year
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:AGGREGATE [FINALIZE]
@@ -875,6 +965,8 @@ group by month, year
 select group_concat(distinct string_col), group_concat(distinct string_col, '-'),
 group_concat(distinct string_col, '---')  from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: group_concat(string_col), group_concat(string_col, '-'), group_concat(string_col, '---')
 |
@@ -884,6 +976,8 @@ group_concat(distinct string_col, '---')  from functional.alltypesagg
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: group_concat:merge(string_col), group_concat:merge(string_col, '-'), group_concat:merge(string_col, '---')
 |
@@ -906,6 +1000,8 @@ group_concat(distinct string_col, '---')  from functional.alltypesagg
 # IMPALA-852: Aggregation only in the HAVING clause.
 select 1 from functional.alltypestiny having count(*) > 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  having: count(*) > 0
@@ -923,6 +1019,8 @@ group by 1
 having count(*) < 150000
 limit 1000000
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |  limit: 1000000
 |
@@ -957,6 +1055,8 @@ select col from (
 where col > 50
 limit 50
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:AGGREGATE [FINALIZE]
 |  output: count:merge(c_custkey)
 |  having: count(c_custkey) > 50
@@ -992,6 +1092,8 @@ select straight_join c_custkey, count(distinct c_custkey)
 from tpch_parquet.orders inner join [shuffle] tpch_parquet.customer on c_custkey = o_custkey
 group by 1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:AGGREGATE [FINALIZE]
@@ -1029,6 +1131,8 @@ group by 1, 2
 having count(*) > 10
 limit 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |  limit: 10
 |


[7/7] incubator-impala git commit: IMPALA-2905: Handle coordinator fragment lifecycle like all others

Posted by he...@apache.org.
IMPALA-2905: Handle coordinator fragment lifecycle like all others

The plan-root fragment instance that runs on the coordinator should be
handled like all others: started via RPC and run asynchronously. Without
this, the fragment requires special-case code throughout the
coordinator, and does not show up in system metrics etc.

This patch adds a new sink type, PlanRootSink, to the root fragment
instance so that the coordinator can pull row batches that are pushed by
the root instance. The coordinator signals completion to the fragment
instance via closing the consumer side of the sink, whereupon the
instance is free to complete.

Since the root instance now runs asynchronously wrt to the coordinator,
we add several coordination methods to allow the coordinator to wait for
a point in the instance's execution to be hit - e.g. to wait until the
instance has been opened.

Done in this patch:

* Add PlanRootSink
* Add coordination to PFE to allow coordinator to observe lifecycle
* Make FragmentMgr a singleton
* Removed dead code from Coordinator::Wait() and elsewhere.
* Moved result output exprs out of QES and into PlanRootSink.
* Remove special-case limit-based teardown of coordinator fragment, and
  supporting functions in PlanFragmentExecutor.
* Simplified lifecycle of PlanFragmentExecutor by separating Open() into
  Open() and Exec(), the latter of which drives the sink by reading
  rows from the plan tree.
* Add child profile to PlanFragmentExecutor to measure time spent in
  each lifecycle phase.
* Removed dependency between InitExecProfiles() and starting root
  fragment.
* Removed mostly dead-code handling of LIMIT 0 queries.
* Ensured that SET returns a result set in all cases.
* Fix test_get_log() HS2 test. Errors are only guaranteed to be visible
  after fetch calls return EOS, but test was assuming this would happen
  after first fetch.

Change-Id: Ibb0064ec2f085fa3a5598ea80894fb489a01e4df
Reviewed-on: http://gerrit.cloudera.org:8080/4402
Tested-by: Internal Jenkins
Reviewed-by: Henry Robinson <he...@cloudera.com>


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

Branch: refs/heads/master
Commit: 9f61397fc4d638aa78b37db2cd5b9c35b6deed94
Parents: 05b91a9
Author: Henry Robinson <he...@cloudera.com>
Authored: Wed Oct 5 11:48:01 2016 -0700
Committer: Henry Robinson <he...@cloudera.com>
Committed: Sun Oct 16 15:55:29 2016 +0000

----------------------------------------------------------------------
 be/src/exec/CMakeLists.txt                      |   1 +
 be/src/exec/data-sink.cc                        |   4 +
 be/src/exec/plan-root-sink.cc                   | 171 ++++++
 be/src/exec/plan-root-sink.h                    | 133 +++++
 be/src/runtime/coordinator.cc                   | 560 +++++++------------
 be/src/runtime/coordinator.h                    | 142 ++---
 be/src/runtime/exec-env.cc                      |  19 +-
 be/src/runtime/exec-env.h                       |   2 +
 be/src/runtime/plan-fragment-executor.cc        | 264 ++++-----
 be/src/runtime/plan-fragment-executor.h         | 149 ++---
 be/src/scheduling/query-schedule.cc             |  32 +-
 be/src/scheduling/query-schedule.h              |  27 +-
 be/src/scheduling/simple-scheduler.cc           |   5 -
 be/src/service/fragment-exec-state.cc           |   6 +-
 be/src/service/fragment-exec-state.h            |   8 +-
 be/src/service/fragment-mgr.cc                  |   8 +-
 be/src/service/impala-beeswax-server.cc         |  52 +-
 be/src/service/impala-hs2-server.cc             |  36 +-
 be/src/service/impala-internal-service.h        |  15 +-
 be/src/service/impala-server.cc                 |   5 +-
 be/src/service/impala-server.h                  |  47 --
 be/src/service/query-exec-state.cc              | 107 +---
 be/src/service/query-exec-state.h               |  11 +-
 be/src/service/query-result-set.h               |  64 +++
 be/src/testutil/in-process-servers.cc           |   4 +
 common/thrift/DataSinks.thrift                  |   7 +-
 .../org/apache/impala/analysis/QueryStmt.java   |   6 +
 .../org/apache/impala/planner/PlanRootSink.java |  39 ++
 .../java/org/apache/impala/planner/Planner.java |   2 +
 .../apache/impala/planner/PlannerContext.java   |   1 +
 .../queries/PlannerTest/aggregation.test        | 104 ++++
 .../queries/PlannerTest/analytic-fns.test       | 118 ++++
 .../PlannerTest/complex-types-file-formats.test |  14 +
 .../queries/PlannerTest/conjunct-ordering.test  |  30 +
 .../queries/PlannerTest/constant.test           |   4 +
 .../queries/PlannerTest/data-source-tables.test |  10 +
 .../PlannerTest/disable-preaggregations.test    |   4 +
 .../queries/PlannerTest/distinct-estimate.test  |   8 +
 .../queries/PlannerTest/distinct.test           |  54 ++
 .../queries/PlannerTest/empty.test              |  58 ++
 .../queries/PlannerTest/hbase.test              | 118 ++++
 .../queries/PlannerTest/hdfs.test               | 226 ++++++++
 .../queries/PlannerTest/implicit-joins.test     |  28 +
 .../queries/PlannerTest/inline-view-limit.test  |  58 ++
 .../queries/PlannerTest/inline-view.test        | 116 ++++
 .../queries/PlannerTest/join-order.test         |  72 +++
 .../queries/PlannerTest/joins.test              | 178 ++++++
 .../queries/PlannerTest/kudu-selectivity.test   |  16 +
 .../queries/PlannerTest/kudu.test               |  32 ++
 .../PlannerTest/mem-limit-broadcast-join.test   |   2 +
 .../queries/PlannerTest/nested-collections.test | 144 +++++
 .../queries/PlannerTest/nested-loop-join.test   |  12 +
 .../queries/PlannerTest/order.test              | 104 ++++
 .../queries/PlannerTest/outer-joins.test        |  54 ++
 .../PlannerTest/partition-key-scans.test        |  38 ++
 .../PlannerTest/predicate-propagation.test      |  90 +++
 .../PlannerTest/runtime-filter-propagation.test |  86 +++
 .../queries/PlannerTest/small-query-opt.test    |  42 ++
 .../queries/PlannerTest/subquery-rewrite.test   | 144 +++++
 .../queries/PlannerTest/topn.test               |  48 ++
 .../queries/PlannerTest/tpcds-all.test          | 146 +++++
 .../queries/PlannerTest/tpch-all.test           | 132 +++++
 .../queries/PlannerTest/tpch-kudu.test          |  44 ++
 .../queries/PlannerTest/tpch-nested.test        |  88 +++
 .../queries/PlannerTest/tpch-views.test         |  44 ++
 .../queries/PlannerTest/union.test              | 198 +++++++
 .../queries/PlannerTest/values.test             |  16 +
 .../queries/PlannerTest/views.test              |  48 ++
 .../queries/PlannerTest/with-clause.test        |  58 ++
 tests/custom_cluster/test_client_ssl.py         |   1 +
 tests/failure/test_failpoints.py                |   2 +-
 tests/hs2/test_hs2.py                           |  16 +-
 tests/hs2/test_json_endpoints.py                |   4 +
 tests/shell/util.py                             |   5 +-
 74 files changed, 3831 insertions(+), 910 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/exec/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/be/src/exec/CMakeLists.txt b/be/src/exec/CMakeLists.txt
index 571198f..fce5c81 100644
--- a/be/src/exec/CMakeLists.txt
+++ b/be/src/exec/CMakeLists.txt
@@ -77,6 +77,7 @@ add_library(Exec
   partitioned-hash-join-builder-ir.cc
   partitioned-hash-join-node.cc
   partitioned-hash-join-node-ir.cc
+  plan-root-sink.cc
   kudu-scanner.cc
   kudu-scan-node.cc
   kudu-table-sink.cc

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/exec/data-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/data-sink.cc b/be/src/exec/data-sink.cc
index b6ec0ee..c95b854 100644
--- a/be/src/exec/data-sink.cc
+++ b/be/src/exec/data-sink.cc
@@ -26,6 +26,7 @@
 #include "exec/hdfs-table-sink.h"
 #include "exec/kudu-table-sink.h"
 #include "exec/kudu-util.h"
+#include "exec/plan-root-sink.h"
 #include "exprs/expr.h"
 #include "gen-cpp/ImpalaInternalService_constants.h"
 #include "gen-cpp/ImpalaInternalService_types.h"
@@ -94,6 +95,9 @@ Status DataSink::CreateDataSink(ObjectPool* pool,
       }
 
       break;
+    case TDataSinkType::PLAN_ROOT_SINK:
+      sink->reset(new PlanRootSink(row_desc, output_exprs, thrift_sink));
+      break;
     default:
       stringstream error_msg;
       map<int, const char*>::const_iterator i =

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/exec/plan-root-sink.cc
----------------------------------------------------------------------
diff --git a/be/src/exec/plan-root-sink.cc b/be/src/exec/plan-root-sink.cc
new file mode 100644
index 0000000..bd73953
--- /dev/null
+++ b/be/src/exec/plan-root-sink.cc
@@ -0,0 +1,171 @@
+// 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/plan-root-sink.h"
+
+#include "exprs/expr-context.h"
+#include "exprs/expr.h"
+#include "runtime/row-batch.h"
+#include "runtime/tuple-row.h"
+#include "service/query-result-set.h"
+
+#include <memory>
+#include <boost/thread/mutex.hpp>
+
+using namespace std;
+using boost::unique_lock;
+using boost::mutex;
+
+namespace impala {
+
+const string PlanRootSink::NAME = "PLAN_ROOT_SINK";
+
+PlanRootSink::PlanRootSink(const RowDescriptor& row_desc,
+    const std::vector<TExpr>& output_exprs, const TDataSink& thrift_sink)
+  : DataSink(row_desc), thrift_output_exprs_(output_exprs) {}
+
+Status PlanRootSink::Prepare(RuntimeState* state, MemTracker* mem_tracker) {
+  RETURN_IF_ERROR(DataSink::Prepare(state, mem_tracker));
+  RETURN_IF_ERROR(
+      Expr::CreateExprTrees(state->obj_pool(), thrift_output_exprs_, &output_expr_ctxs_));
+  RETURN_IF_ERROR(
+      Expr::Prepare(output_expr_ctxs_, state, row_desc_, expr_mem_tracker_.get()));
+
+  return Status::OK();
+}
+
+Status PlanRootSink::Open(RuntimeState* state) {
+  RETURN_IF_ERROR(Expr::Open(output_expr_ctxs_, state));
+  return Status::OK();
+}
+
+namespace {
+
+/// Validates that all collection-typed slots in the given batch are set to NULL.
+/// See SubplanNode for details on when collection-typed slots are set to NULL.
+/// TODO: This validation will become obsolete when we can return collection values.
+/// We will then need a different mechanism to assert the correct behavior of the
+/// SubplanNode with respect to setting collection-slots to NULL.
+void ValidateCollectionSlots(const RowDescriptor& row_desc, RowBatch* batch) {
+#ifndef NDEBUG
+  if (!row_desc.HasVarlenSlots()) return;
+  for (int i = 0; i < batch->num_rows(); ++i) {
+    TupleRow* row = batch->GetRow(i);
+    for (int j = 0; j < row_desc.tuple_descriptors().size(); ++j) {
+      const TupleDescriptor* tuple_desc = row_desc.tuple_descriptors()[j];
+      if (tuple_desc->collection_slots().empty()) continue;
+      for (int k = 0; k < tuple_desc->collection_slots().size(); ++k) {
+        const SlotDescriptor* slot_desc = tuple_desc->collection_slots()[k];
+        int tuple_idx = row_desc.GetTupleIdx(slot_desc->parent()->id());
+        const Tuple* tuple = row->GetTuple(tuple_idx);
+        if (tuple == NULL) continue;
+        DCHECK(tuple->IsNull(slot_desc->null_indicator_offset()));
+      }
+    }
+  }
+#endif
+}
+}
+
+Status PlanRootSink::Send(RuntimeState* state, RowBatch* batch) {
+  ValidateCollectionSlots(row_desc_, batch);
+  int current_batch_row = 0;
+  do {
+    unique_lock<mutex> l(lock_);
+    while (results_ == nullptr && !consumer_done_) sender_cv_.wait(l);
+    if (consumer_done_ || batch == nullptr) {
+      eos_ = true;
+      return Status::OK();
+    }
+
+    // Otherwise the consumer is ready. Fill out the rows.
+    DCHECK(results_ != nullptr);
+    // List of expr values to hold evaluated rows from the query
+    vector<void*> result_row;
+    result_row.resize(output_expr_ctxs_.size());
+
+    // List of scales for floating point values in result_row
+    vector<int> scales;
+    scales.resize(result_row.size());
+
+    int num_to_fetch = batch->num_rows() - current_batch_row;
+    if (num_rows_requested_ > 0) num_to_fetch = min(num_to_fetch, num_rows_requested_);
+    for (int i = 0; i < num_to_fetch; ++i) {
+      TupleRow* row = batch->GetRow(current_batch_row);
+      GetRowValue(row, &result_row, &scales);
+      RETURN_IF_ERROR(results_->AddOneRow(result_row, scales));
+      ++current_batch_row;
+    }
+    // Signal the consumer.
+    results_ = nullptr;
+    ExprContext::FreeLocalAllocations(output_expr_ctxs_);
+    consumer_cv_.notify_all();
+  } while (current_batch_row < batch->num_rows());
+  return Status::OK();
+}
+
+Status PlanRootSink::FlushFinal(RuntimeState* state) {
+  unique_lock<mutex> l(lock_);
+  sender_done_ = true;
+  eos_ = true;
+  consumer_cv_.notify_all();
+  return Status::OK();
+}
+
+void PlanRootSink::Close(RuntimeState* state) {
+  unique_lock<mutex> l(lock_);
+  // No guarantee that FlushFinal() has been called, so need to mark sender_done_ here as
+  // well.
+  sender_done_ = true;
+  consumer_cv_.notify_all();
+  // Wait for consumer to be done, in case sender tries to tear-down this sink while the
+  // sender is still reading from it.
+  while (!consumer_done_) sender_cv_.wait(l);
+  Expr::Close(output_expr_ctxs_, state);
+  DataSink::Close(state);
+}
+
+void PlanRootSink::CloseConsumer() {
+  unique_lock<mutex> l(lock_);
+  consumer_done_ = true;
+  sender_cv_.notify_all();
+}
+
+Status PlanRootSink::GetNext(
+    RuntimeState* state, QueryResultSet* results, int num_results, bool* eos) {
+  unique_lock<mutex> l(lock_);
+  DCHECK(!consumer_done_);
+
+  results_ = results;
+  num_rows_requested_ = num_results;
+  sender_cv_.notify_all();
+
+  while (!eos_ && results_ != nullptr && !sender_done_) consumer_cv_.wait(l);
+  *eos = eos_;
+  RETURN_IF_ERROR(state->CheckQueryState());
+  return Status::OK();
+}
+
+void PlanRootSink::GetRowValue(
+    TupleRow* row, vector<void*>* result, vector<int>* scales) {
+  DCHECK(result->size() >= output_expr_ctxs_.size());
+  for (int i = 0; i < output_expr_ctxs_.size(); ++i) {
+    (*result)[i] = output_expr_ctxs_[i]->GetValue(row);
+    (*scales)[i] = output_expr_ctxs_[i]->root()->output_scale();
+  }
+}
+}

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/exec/plan-root-sink.h
----------------------------------------------------------------------
diff --git a/be/src/exec/plan-root-sink.h b/be/src/exec/plan-root-sink.h
new file mode 100644
index 0000000..cc7c045
--- /dev/null
+++ b/be/src/exec/plan-root-sink.h
@@ -0,0 +1,133 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#ifndef IMPALA_EXEC_PLAN_ROOT_SINK_H
+#define IMPALA_EXEC_PLAN_ROOT_SINK_H
+
+#include "exec/data-sink.h"
+
+#include <boost/thread/condition_variable.hpp>
+
+namespace impala {
+
+class TupleRow;
+class RowBatch;
+class QueryResultSet;
+class ExprContext;
+
+/// Sink which manages the handoff between a 'sender' (a fragment instance) that produces
+/// batches by calling Send(), and a 'consumer' (e.g. the coordinator) which consumes rows
+/// formed by computing a set of output expressions over the input batches, by calling
+/// GetNext(). Send() and GetNext() are called concurrently.
+///
+/// The consumer calls GetNext() with a QueryResultSet and a requested fetch
+/// size. GetNext() shares these fields with Send(), and then signals Send() to begin
+/// populating the result set. GetNext() returns when either the sender has sent all of
+/// its rows, or the requested fetch size has been satisfied.
+///
+/// Send() fills in as many rows as are requested from the current batch. When the batch
+/// is exhausted - which may take several calls to GetNext() - control is returned to the
+/// sender to produce another row batch.
+///
+/// Consumers must call CloseConsumer() when finished to allow the fragment to shut
+/// down. Senders must call Close() to signal to the consumer that no more batches will be
+/// produced.
+///
+/// The sink is thread safe up to a single producer and single consumer.
+///
+/// TODO: The consumer drives the sender in lock-step with GetNext() calls, forcing a
+/// context-switch on every invocation. Measure the impact of this, and consider moving to
+/// a fully asynchronous implementation with a queue to manage buffering between sender
+/// and consumer. See IMPALA-4268.
+class PlanRootSink : public DataSink {
+ public:
+  PlanRootSink(const RowDescriptor& row_desc, const std::vector<TExpr>& output_exprs,
+      const TDataSink& thrift_sink);
+
+  virtual std::string GetName() { return NAME; }
+
+  virtual Status Prepare(RuntimeState* state, MemTracker* tracker);
+
+  virtual Status Open(RuntimeState* state);
+
+  /// Sends a new batch. Ownership of 'batch' remains with the sender. Blocks until the
+  /// consumer has consumed 'batch' by calling GetNext().
+  virtual Status Send(RuntimeState* state, RowBatch* batch);
+
+  /// Sets eos and notifies consumer.
+  virtual Status FlushFinal(RuntimeState* state);
+
+  /// To be called by sender only. Signals to the consumer that no more batches will be
+  /// produced, then blocks until the consumer calls CloseConsumer().
+  virtual void Close(RuntimeState* state);
+
+  /// Populates 'result_set' with up to 'num_rows' rows produced by the fragment instance
+  /// that calls Send(). *eos is set to 'true' when there are no more rows to consume.
+  Status GetNext(
+      RuntimeState* state, QueryResultSet* result_set, int num_rows, bool* eos);
+
+  /// Signals to the producer that the sink will no longer be used. It's an error to call
+  /// GetNext() after this returns. May be called more than once; only the first call has
+  /// any effect.
+  void CloseConsumer();
+
+  static const std::string NAME;
+
+ private:
+  /// Protects all members, including the condition variables.
+  boost::mutex lock_;
+
+  /// Waited on by the sender only. Signalled when the consumer has written results_ and
+  /// num_rows_requested_, and so the sender may begin satisfying that request for rows
+  /// from its current batch. Also signalled when CloseConsumer() is called, to unblock
+  /// the sender.
+  boost::condition_variable sender_cv_;
+
+  /// Waited on by the consumer only. Signalled when the sender has finished serving a
+  /// request for rows. Also signalled by Close() and FlushFinal() to signal to the
+  /// consumer that no more rows are coming.
+  boost::condition_variable consumer_cv_;
+
+  /// Signals to producer that the consumer is done, and the sink may be torn down.
+  bool consumer_done_ = false;
+
+  /// Signals to consumer that the sender is done, and that there are no more row batches
+  /// to consume.
+  bool sender_done_ = false;
+
+  /// The current result set passed to GetNext(), to fill in Send(). Not owned by this
+  /// sink. Reset to nullptr after Send() completes the request to signal to the consumer
+  /// that it can return.
+  QueryResultSet* results_ = nullptr;
+
+  /// Set by GetNext() to indicate to Send() how many rows it should write to results_.
+  int num_rows_requested_ = 0;
+
+  /// Set to true in Send() and FlushFinal() when the Sink() has finished producing rows.
+  bool eos_ = false;
+
+  /// Output expressions to map plan row batches onto result set rows.
+  std::vector<TExpr> thrift_output_exprs_;
+  std::vector<ExprContext*> output_expr_ctxs_;
+
+  /// Writes a single row into 'result' and 'scales' by evaluating output_expr_ctxs_ over
+  /// 'row'.
+  void GetRowValue(TupleRow* row, std::vector<void*>* result, std::vector<int>* scales);
+};
+}
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/runtime/coordinator.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.cc b/be/src/runtime/coordinator.cc
index df4ad7b..4214d4d 100644
--- a/be/src/runtime/coordinator.cc
+++ b/be/src/runtime/coordinator.cc
@@ -39,22 +39,28 @@
 #include <errno.h>
 
 #include "common/logging.h"
-#include "exprs/expr.h"
 #include "exec/data-sink.h"
+#include "exec/plan-root-sink.h"
+#include "exec/scan-node.h"
+#include "gen-cpp/Frontend_types.h"
+#include "gen-cpp/ImpalaInternalService.h"
+#include "gen-cpp/ImpalaInternalService_constants.h"
+#include "gen-cpp/ImpalaInternalService_types.h"
+#include "gen-cpp/Partitions_types.h"
+#include "gen-cpp/PlanNodes_types.h"
+#include "runtime/backend-client.h"
 #include "runtime/client-cache.h"
-#include "runtime/data-stream-sender.h"
 #include "runtime/data-stream-mgr.h"
+#include "runtime/data-stream-sender.h"
 #include "runtime/exec-env.h"
 #include "runtime/hdfs-fs-cache.h"
 #include "runtime/mem-tracker.h"
+#include "runtime/parallel-executor.h"
 #include "runtime/plan-fragment-executor.h"
 #include "runtime/row-batch.h"
-#include "runtime/backend-client.h"
-#include "runtime/parallel-executor.h"
 #include "runtime/tuple-row.h"
 #include "scheduling/scheduler.h"
-#include "exec/data-sink.h"
-#include "exec/scan-node.h"
+#include "service/fragment-exec-state.h"
 #include "util/bloom-filter.h"
 #include "util/container-util.h"
 #include "util/counting-barrier.h"
@@ -67,12 +73,6 @@
 #include "util/summary-util.h"
 #include "util/table-printer.h"
 #include "util/uid-util.h"
-#include "gen-cpp/ImpalaInternalService.h"
-#include "gen-cpp/ImpalaInternalService_types.h"
-#include "gen-cpp/Frontend_types.h"
-#include "gen-cpp/PlanNodes_types.h"
-#include "gen-cpp/Partitions_types.h"
-#include "gen-cpp/ImpalaInternalService_constants.h"
 
 #include "common/names.h"
 
@@ -240,8 +240,8 @@ class Coordinator::FragmentInstanceState {
   mutex lock_;
 
   /// If the status indicates an error status, execution of this fragment has either been
-  /// aborted by the remote impalad (which then reported the error) or cancellation has
-  /// been initiated; either way, execution must not be cancelled
+  /// aborted by the executing impalad (which then reported the error) or cancellation has
+  /// been initiated; either way, execution must not be cancelled.
   Status status_;
 
   /// If true, ExecPlanFragment() rpc has been sent.
@@ -377,15 +377,13 @@ Coordinator::Coordinator(const QuerySchedule& schedule, ExecEnv* exec_env,
     exec_env_(exec_env),
     has_called_wait_(false),
     returned_all_results_(false),
-    executor_(NULL), // Set in Prepare()
     query_mem_tracker_(), // Set in Exec()
     num_remaining_fragment_instances_(0),
     obj_pool_(new ObjectPool()),
     query_events_(events),
     filter_routing_table_complete_(false),
-    filter_mode_(schedule.query_options().runtime_filter_mode),
-    torn_down_(false) {
-}
+    filter_mode_(schedule_.query_options().runtime_filter_mode),
+    torn_down_(false) {}
 
 Coordinator::~Coordinator() {
   DCHECK(torn_down_) << "TearDown() must be called before Coordinator is destroyed";
@@ -448,14 +446,15 @@ static void ProcessQueryOptions(
       << "because nodes cannot be cancelled in Close()";
 }
 
-Status Coordinator::Exec(vector<ExprContext*>* output_expr_ctxs) {
+Status Coordinator::Exec() {
   const TQueryExecRequest& request = schedule_.request();
   DCHECK(request.fragments.size() > 0 || request.mt_plan_exec_info.size() > 0);
+
   needs_finalization_ = request.__isset.finalize_params;
   if (needs_finalization_) finalize_params_ = request.finalize_params;
 
   VLOG_QUERY << "Exec() query_id=" << schedule_.query_id()
-      << " stmt=" << request.query_ctx.request.stmt;
+             << " stmt=" << request.query_ctx.request.stmt;
   stmt_type_ = request.stmt_type;
   query_id_ = schedule_.query_id();
   desc_tbl_ = request.desc_tbl;
@@ -468,18 +467,6 @@ Status Coordinator::Exec(vector<ExprContext*>* output_expr_ctxs) {
 
   SCOPED_TIMER(query_profile_->total_time_counter());
 
-  // After the coordinator fragment is started, it may call UpdateFilter() asynchronously,
-  // which waits on this barrier for completion.
-  // TODO: remove special treatment of coord fragment
-  int num_remote_instances = schedule_.GetNumRemoteFInstances();
-  if (num_remote_instances > 0) {
-    exec_complete_barrier_.reset(new CountingBarrier(num_remote_instances));
-  }
-  num_remaining_fragment_instances_ = num_remote_instances;
-
-  // TODO: move initial setup into a separate function; right now part of it
-  // (InitExecProfile()) depends on the coordinator fragment having been started
-
   // initialize progress updater
   const string& str = Substitute("Query $0", PrintId(query_id_));
   progress_.Init(str, schedule_.num_scan_ranges());
@@ -489,64 +476,51 @@ Status Coordinator::Exec(vector<ExprContext*>* output_expr_ctxs) {
   // execution at Impala daemons where it hasn't even started
   lock_guard<mutex> l(lock_);
 
-  bool has_coordinator_fragment = schedule_.GetCoordFragment() != NULL;
-  if (has_coordinator_fragment) {
-    // Start this before starting any more plan
-    // fragments, otherwise they start sending data before the local exchange node had a
-    // chance to register with the stream mgr.
-    // TODO: This is no longer necessary (see IMPALA-1599). Consider starting all
-    // fragments in the same way with no coordinator special case.
-    RETURN_IF_ERROR(PrepareCoordFragment(output_expr_ctxs));
-  } else {
-    // The coordinator instance may require a query mem tracker even if there is no
-    // coordinator fragment. For example, result-caching tracks memory via the query mem
-    // tracker.
-    // If there is a fragment, the fragment executor created above initializes the query
-    // mem tracker. If not, the query mem tracker is created here.
-    int64_t query_limit = -1;
-    if (query_ctx_.request.query_options.__isset.mem_limit &&
-        query_ctx_.request.query_options.mem_limit > 0) {
-      query_limit = query_ctx_.request.query_options.mem_limit;
-    }
-    MemTracker* pool_tracker = MemTracker::GetRequestPoolMemTracker(
-        schedule_.request_pool(), exec_env_->process_mem_tracker());
-    query_mem_tracker_ =
-        MemTracker::GetQueryMemTracker(query_id_, query_limit, pool_tracker);
-
-    executor_.reset(NULL);
-  }
+  // The coordinator may require a query mem tracker for result-caching, which tracks
+  // memory via the query mem tracker.
+  int64_t query_limit = -1;
+  if (query_ctx_.request.query_options.__isset.mem_limit
+      && query_ctx_.request.query_options.mem_limit > 0) {
+    query_limit = query_ctx_.request.query_options.mem_limit;
+  }
+  MemTracker* pool_tracker = MemTracker::GetRequestPoolMemTracker(
+      schedule_.request_pool(), exec_env_->process_mem_tracker());
+  query_mem_tracker_ =
+      MemTracker::GetQueryMemTracker(query_id_, query_limit, pool_tracker);
+  DCHECK(query_mem_tracker() != nullptr);
   filter_mem_tracker_.reset(
       new MemTracker(-1, "Runtime Filter (Coordinator)", query_mem_tracker(), false));
 
-  // initialize execution profile structures
+  // Initialize the execution profile structures.
   bool is_mt_execution = request.query_ctx.request.query_options.mt_dop > 0;
   if (is_mt_execution) {
     MtInitExecProfiles();
     MtInitExecSummary();
+    MtStartFInstances();
   } else {
     InitExecProfile(request);
+    StartFragments();
   }
 
-  if (num_remote_instances > 0) {
-    // pre-size fragment_instance_states_ in order to directly address by instance idx
-    // when creating FragmentInstanceStates (instead of push_back())
-    int num_fragment_instances = schedule_.GetTotalFInstances();
-    DCHECK_GT(num_fragment_instances, 0);
-    fragment_instance_states_.resize(num_fragment_instances);
+  RETURN_IF_ERROR(FinishInstanceStartup());
 
-    if (is_mt_execution) {
-      MtStartRemoteFInstances();
-    } else {
-      StartRemoteFragments();
-    }
-    RETURN_IF_ERROR(FinishRemoteInstanceStartup());
+  // Grab executor and wait until Prepare() has finished so that runtime state etc. will
+  // be set up.
+  if (schedule_.GetCoordFragment() != nullptr) {
+    // Coordinator fragment instance has same ID as query.
+    shared_ptr<FragmentMgr::FragmentExecState> root_fragment_instance =
+        ExecEnv::GetInstance()->fragment_mgr()->GetFragmentExecState(query_id_);
+    DCHECK(root_fragment_instance.get() != nullptr);
+    executor_ = root_fragment_instance->executor();
 
-    // If we have a coordinator fragment and remote fragments (the common case), release
-    // the thread token on the coordinator fragment. This fragment spends most of the time
-    // waiting and doing very little work. Holding on to the token causes underutilization
-    // of the machine. If there are 12 queries on this node, that's 12 tokens reserved for
-    // no reason.
-    if (has_coordinator_fragment) executor_->ReleaseThreadToken();
+    // When WaitForPrepare() returns OK(), the executor's root sink will be set up. At
+    // that point, the coordinator must be sure to call root_sink()->CloseConsumer(); the
+    // fragment instance's executor will not complete until that point.
+    // TODO: Consider moving this to Wait().
+    Status prepare_status = executor_->WaitForPrepare();
+    root_sink_ = executor_->root_sink();
+    RETURN_IF_ERROR(prepare_status);
+    DCHECK(root_sink_ != nullptr);
   }
 
   PrintFragmentInstanceInfo();
@@ -610,60 +584,14 @@ void Coordinator::UpdateFilterRoutingTable(const vector<TPlanNode>& plan_nodes,
   }
 }
 
-Status Coordinator::PrepareCoordFragment(vector<ExprContext*>* output_expr_ctxs) {
-  const TQueryExecRequest& request = schedule_.request();
-  bool is_mt_execution = request.query_ctx.request.query_options.mt_dop > 0;
-  if (!is_mt_execution && filter_mode_ != TRuntimeFilterMode::OFF) {
-    UpdateFilterRoutingTable(schedule_.GetCoordFragment()->plan.nodes, 1, 0);
-    if (schedule_.GetNumFragmentInstances() == 0) MarkFilterRoutingTableComplete();
-  }
+void Coordinator::StartFragments() {
+  int num_fragment_instances = schedule_.GetNumFragmentInstances();
+  DCHECK_GT(num_fragment_instances, 0);
 
-  // create rpc params and FragmentInstanceState for the coordinator fragment
-  TExecPlanFragmentParams rpc_params;
-  FragmentInstanceState* coord_state = nullptr;
-  if (is_mt_execution) {
-    const FInstanceExecParams& coord_params = schedule_.GetCoordInstanceExecParams();
-    MtSetExecPlanFragmentParams(coord_params, &rpc_params);
-    coord_state = obj_pool()->Add(
-        new FragmentInstanceState(coord_params, obj_pool()));
-  } else {
-    const TPlanFragment& coord_fragment = *schedule_.GetCoordFragment();
-    SetExecPlanFragmentParams(
-        coord_fragment, schedule_.exec_params()[0], 0, &rpc_params);
-    coord_state = obj_pool()->Add(
-        new FragmentInstanceState(
-          coord_fragment.idx, schedule_.exec_params()[0], 0, obj_pool()));
-    // apparently this was never called for the coordinator fragment
-    // TODO: fix this
-    //exec_state->ComputeTotalSplitSize(
-        //rpc_params.fragment_instance_ctx.per_node_scan_ranges);
-  }
-  // register state before calling Prepare(), in case it fails
-  DCHECK_EQ(GetInstanceIdx(coord_state->fragment_instance_id()), 0);
-  fragment_instance_states_.push_back(coord_state);
-  DCHECK(coord_state != nullptr);
-  DCHECK_EQ(fragment_instance_states_.size(), 1);
-  executor_.reset(new PlanFragmentExecutor(
-      exec_env_, PlanFragmentExecutor::ReportStatusCallback()));
-  RETURN_IF_ERROR(executor_->Prepare(rpc_params));
-  coord_state->set_profile(executor_->profile());
-
-  // Prepare output_expr_ctxs before optimizing the LLVM module. The other exprs of this
-  // coordinator fragment have been prepared in executor_->Prepare().
-  DCHECK(output_expr_ctxs != NULL);
-  RETURN_IF_ERROR(Expr::CreateExprTrees(
-      runtime_state()->obj_pool(), schedule_.GetCoordFragment()->output_exprs,
-      output_expr_ctxs));
-  MemTracker* output_expr_tracker = runtime_state()->obj_pool()->Add(new MemTracker(
-      -1, "Output exprs", runtime_state()->instance_mem_tracker(), false));
-  RETURN_IF_ERROR(Expr::Prepare(
-      *output_expr_ctxs, runtime_state(), row_desc(), output_expr_tracker));
+  fragment_instance_states_.resize(num_fragment_instances);
+  exec_complete_barrier_.reset(new CountingBarrier(num_fragment_instances));
+  num_remaining_fragment_instances_ = num_fragment_instances;
 
-  return Status::OK();
-}
-
-void Coordinator::StartRemoteFragments() {
-  int num_fragment_instances = schedule_.GetNumFragmentInstances();
   DebugOptions debug_options;
   ProcessQueryOptions(schedule_.query_options(), &debug_options);
   const TQueryExecRequest& request = schedule_.request();
@@ -671,19 +599,14 @@ void Coordinator::StartRemoteFragments() {
   VLOG_QUERY << "starting " << num_fragment_instances << " fragment instances for query "
              << query_id_;
   query_events_->MarkEvent(
-      Substitute("Ready to start $0 remote fragment instances", num_fragment_instances));
+      Substitute("Ready to start $0 fragments", num_fragment_instances));
 
-  bool has_coordinator_fragment =
-      request.fragments[0].partition.type == TPartitionType::UNPARTITIONED;
-  int instance_state_idx = has_coordinator_fragment ? 1 : 0;
-  int first_remote_fragment_idx = has_coordinator_fragment ? 1 : 0;
+  int instance_state_idx = 0;
   if (filter_mode_ != TRuntimeFilterMode::OFF) {
-    // Populate the runtime filter routing table. This should happen before starting
-    // the remote fragments.
-    // This code anticipates the indices of the instance states created later on in
-    // ExecRemoteFragment()
-    for (int fragment_idx = first_remote_fragment_idx;
-         fragment_idx < request.fragments.size(); ++fragment_idx) {
+    // Populate the runtime filter routing table. This should happen before starting the
+    // fragment instances. This code anticipates the indices of the instance states
+    // created later on in ExecRemoteFragment()
+    for (int fragment_idx = 0; fragment_idx < request.fragments.size(); ++fragment_idx) {
       const FragmentExecParams& params = schedule_.exec_params()[fragment_idx];
       int num_hosts = params.hosts.size();
       DCHECK_GT(num_hosts, 0);
@@ -697,8 +620,7 @@ void Coordinator::StartRemoteFragments() {
   int num_instances = 0;
   // Start one fragment instance per fragment per host (number of hosts running each
   // fragment may not be constant).
-  for (int fragment_idx = first_remote_fragment_idx;
-       fragment_idx < request.fragments.size(); ++fragment_idx) {
+  for (int fragment_idx = 0; fragment_idx < request.fragments.size(); ++fragment_idx) {
     const FragmentExecParams& params = schedule_.exec_params()[fragment_idx];
     int num_hosts = params.hosts.size();
     DCHECK_GT(num_hosts, 0);
@@ -719,11 +641,17 @@ void Coordinator::StartRemoteFragments() {
   }
   exec_complete_barrier_->Wait();
   query_events_->MarkEvent(
-      Substitute("All $0 remote fragments instances started", num_instances));
+      Substitute("All $0 fragments instances started", num_instances));
 }
 
-void Coordinator::MtStartRemoteFInstances() {
+void Coordinator::MtStartFInstances() {
   int num_fragment_instances = schedule_.GetNumFragmentInstances();
+  DCHECK_GT(num_fragment_instances, 0);
+
+  fragment_instance_states_.resize(num_fragment_instances);
+  exec_complete_barrier_.reset(new CountingBarrier(num_fragment_instances));
+  num_remaining_fragment_instances_ = num_fragment_instances;
+
   DebugOptions debug_options;
   ProcessQueryOptions(schedule_.query_options(), &debug_options);
   const TQueryExecRequest& request = schedule_.request();
@@ -731,7 +659,7 @@ void Coordinator::MtStartRemoteFInstances() {
   VLOG_QUERY << "starting " << num_fragment_instances << " fragment instances for query "
              << query_id_;
   query_events_->MarkEvent(
-      Substitute("Ready to start $0 remote fragment instances", num_fragment_instances));
+      Substitute("Ready to start $0 fragment instances", num_fragment_instances));
 
   // TODO: populate the runtime filter routing table
   // this requires local aggregation of filters prior to sending
@@ -739,7 +667,6 @@ void Coordinator::MtStartRemoteFInstances() {
 
   int num_instances = 0;
   for (const MtFragmentExecParams& fragment_params: schedule_.mt_fragment_exec_params()) {
-    if (fragment_params.is_coord_fragment) continue;
     for (int i = 0; i < fragment_params.instance_exec_params.size();
         ++i, ++num_instances) {
       const FInstanceExecParams& instance_params =
@@ -760,10 +687,10 @@ void Coordinator::MtStartRemoteFInstances() {
   VLOG_QUERY << "started " << num_fragment_instances << " fragment instances for query "
       << query_id_;
   query_events_->MarkEvent(
-      Substitute("All $0 remote fragment instances started", num_instances));
+      Substitute("All $0 fragment instances started", num_instances));
 }
 
-Status Coordinator::FinishRemoteInstanceStartup() {
+Status Coordinator::FinishInstanceStartup() {
   Status status = Status::OK();
   const TMetricDef& def =
       MakeTMetricDef("fragment-latencies", TMetricKind::HISTOGRAM, TUnit::TIME_MS);
@@ -868,7 +795,7 @@ Status Coordinator::UpdateStatus(const Status& status, const TUniqueId& instance
   {
     lock_guard<mutex> l(lock_);
 
-    // The query is done and we are just waiting for remote fragments to clean up.
+    // The query is done and we are just waiting for fragment instances to clean up.
     // Ignore their cancelled updates.
     if (returned_all_results_ && status.IsCancelled()) return query_status_;
 
@@ -1187,7 +1114,8 @@ Status Coordinator::WaitForAllInstances() {
   if (query_status_.ok()) {
     VLOG_QUERY << "All fragment instances finished successfully.";
   } else {
-    VLOG_QUERY << "All fragment instances finished due to one or more errors.";
+    VLOG_QUERY << "All fragment instances finished due to one or more errors. "
+               << query_status_.GetDetail();
   }
 
   return query_status_;
@@ -1198,143 +1126,84 @@ Status Coordinator::Wait() {
   SCOPED_TIMER(query_profile_->total_time_counter());
   if (has_called_wait_) return Status::OK();
   has_called_wait_ = true;
-  Status return_status = Status::OK();
-  if (executor_.get() != NULL) {
-    // Open() may block
-    return_status = UpdateStatus(executor_->Open(),
-        runtime_state()->fragment_instance_id(), FLAGS_hostname);
-
-    if (return_status.ok()) {
-      // If the coordinator fragment has a sink, it will have finished executing at this
-      // point.  It's safe therefore to copy the set of files to move and updated
-      // partitions into the query-wide set.
-      RuntimeState* state = runtime_state();
-      DCHECK(state != NULL);
-
-      // No other instances should have updated these structures if the coordinator has a
-      // fragment. (Instances have a sink only if the coordinator does not)
-      DCHECK_EQ(files_to_move_.size(), 0);
-      DCHECK_EQ(per_partition_status_.size(), 0);
-
-      // Because there are no other updates, safe to copy the maps rather than merge them.
-      files_to_move_ = *state->hdfs_files_to_move();
-      per_partition_status_ = *state->per_partition_status();
-    }
-  } else {
-    // Query finalization can only happen when all instances have reported
-    // relevant state. They only have relevant state to report in the parallel
-    // INSERT case, otherwise all the relevant state is from the coordinator
-    // fragment which will be available after Open() returns.
-    // Ignore the returned status if finalization is required., since FinalizeQuery() will
-    // pick it up and needs to execute regardless.
-    Status status = WaitForAllInstances();
-    if (!needs_finalization_ && !status.ok()) return status;
-  }
 
-  // Query finalization is required only for HDFS table sinks
-  if (needs_finalization_) {
-    RETURN_IF_ERROR(FinalizeQuery());
+  if (stmt_type_ == TStmtType::QUERY) {
+    DCHECK(executor_ != nullptr);
+    return UpdateStatus(executor_->WaitForOpen(), runtime_state()->fragment_instance_id(),
+        FLAGS_hostname);
   }
 
-  if (stmt_type_ == TStmtType::DML) {
-    query_profile_->AddInfoString("Insert Stats",
-        DataSink::OutputInsertStats(per_partition_status_, "\n"));
-    // For DML queries, when Wait is done, the query is complete.  Report aggregate
-    // query profiles at this point.
-    // TODO: make sure ReportQuerySummary gets called on error
-    ReportQuerySummary();
-  }
+  DCHECK_EQ(stmt_type_, TStmtType::DML);
+  // Query finalization can only happen when all backends have reported
+  // relevant state. They only have relevant state to report in the parallel
+  // INSERT case, otherwise all the relevant state is from the coordinator
+  // fragment which will be available after Open() returns.
+  // Ignore the returned status if finalization is required., since FinalizeQuery() will
+  // pick it up and needs to execute regardless.
+  Status status = WaitForAllInstances();
+  if (!needs_finalization_ && !status.ok()) return status;
 
-  if (filter_routing_table_.size() > 0) {
-    query_profile_->AddInfoString("Final filter table", FilterDebugString());
-  }
+  // Query finalization is required only for HDFS table sinks
+  if (needs_finalization_) RETURN_IF_ERROR(FinalizeQuery());
 
-  return return_status;
+  query_profile_->AddInfoString(
+      "Insert Stats", DataSink::OutputInsertStats(per_partition_status_, "\n"));
+  // For DML queries, when Wait is done, the query is complete.  Report aggregate
+  // query profiles at this point.
+  // TODO: make sure ReportQuerySummary gets called on error
+  ReportQuerySummary();
+
+  return status;
 }
 
-Status Coordinator::GetNext(RowBatch** batch, RuntimeState* state) {
+Status Coordinator::GetNext(QueryResultSet* results, int max_rows, bool* eos) {
   VLOG_ROW << "GetNext() query_id=" << query_id_;
   DCHECK(has_called_wait_);
   SCOPED_TIMER(query_profile_->total_time_counter());
 
-  if (executor_.get() == NULL) {
-    // If there is no local fragment, we produce no output, and execution will
-    // have finished after Wait.
-    *batch = NULL;
-    return GetStatus();
-  }
-
-  // do not acquire lock_ here, otherwise we could block and prevent an async
-  // Cancel() from proceeding
-  Status status = executor_->GetNext(batch);
+  DCHECK(root_sink_ != nullptr)
+      << "GetNext() called without result sink. Perhaps Prepare() failed and was not "
+      << "checked?";
+  Status status = root_sink_->GetNext(runtime_state(), results, max_rows, eos);
 
   // if there was an error, we need to return the query's error status rather than
   // the status we just got back from the local executor (which may well be CANCELLED
   // in that case).  Coordinator fragment failed in this case so we log the query_id.
-  RETURN_IF_ERROR(UpdateStatus(status, runtime_state()->fragment_instance_id(),
-      FLAGS_hostname));
+  RETURN_IF_ERROR(
+      UpdateStatus(status, runtime_state()->fragment_instance_id(), FLAGS_hostname));
 
-  if (*batch == NULL) {
+  if (*eos) {
     returned_all_results_ = true;
-    if (executor_->ReachedLimit()) {
-      // We've reached the query limit, cancel the remote fragments.  The
-      // Exchange node on our fragment is no longer receiving rows so the
-      // remote fragments must be explicitly cancelled.
-      CancelRemoteFragments();
-      RuntimeState* state = runtime_state();
-      if (state != NULL) {
-        // Cancel the streams receiving batches.  The exchange nodes that would
-        // normally read from the streams are done.
-        state->stream_mgr()->Cancel(state->fragment_instance_id());
-      }
-    }
-
-    // Don't return final NULL until all instances have completed.
-    // GetNext must wait for all instances to complete before
-    // ultimately signalling the end of execution via a NULL
-    // batch. After NULL is returned, the coordinator may tear down
-    // query state, and perform post-query finalization which might
-    // depend on the reports from all instances.
+    // Trigger tear-down of coordinator fragment by closing the consumer. Must do before
+    // WaitForAllInstances().
+    root_sink_->CloseConsumer();
+    root_sink_ = nullptr;
+
+    // Don't return final NULL until all instances have completed.  GetNext must wait for
+    // all instances to complete before ultimately signalling the end of execution via a
+    // NULL batch. After NULL is returned, the coordinator may tear down query state, and
+    // perform post-query finalization which might depend on the reports from all
+    // instances.
+    //
+    // TODO: Waiting should happen in TearDown() (and then we wouldn't need to call
+    // CloseConsumer() here). See IMPALA-4275 for details.
     RETURN_IF_ERROR(WaitForAllInstances());
     if (query_status_.ok()) {
       // If the query completed successfully, report aggregate query profiles.
       ReportQuerySummary();
     }
-  } else {
-#ifndef NDEBUG
-    ValidateCollectionSlots(*batch);
-#endif
   }
 
   return Status::OK();
 }
 
-void Coordinator::ValidateCollectionSlots(RowBatch* batch) {
-  const RowDescriptor& row_desc = executor_->row_desc();
-  if (!row_desc.HasVarlenSlots()) return;
-  for (int i = 0; i < batch->num_rows(); ++i) {
-    TupleRow* row = batch->GetRow(i);
-    for (int j = 0; j < row_desc.tuple_descriptors().size(); ++j) {
-      const TupleDescriptor* tuple_desc = row_desc.tuple_descriptors()[j];
-      if (tuple_desc->collection_slots().empty()) continue;
-      for (int k = 0; k < tuple_desc->collection_slots().size(); ++k) {
-        const SlotDescriptor* slot_desc = tuple_desc->collection_slots()[k];
-        int tuple_idx = row_desc.GetTupleIdx(slot_desc->parent()->id());
-        const Tuple* tuple = row->GetTuple(tuple_idx);
-        if (tuple == NULL) continue;
-        DCHECK(tuple->IsNull(slot_desc->null_indicator_offset()));
-      }
-    }
-  }
-}
-
 void Coordinator::PrintFragmentInstanceInfo() {
   for (FragmentInstanceState* state: fragment_instance_states_) {
     SummaryStats& acc = fragment_profiles_[state->fragment_idx()].bytes_assigned;
     acc(state->total_split_size());
   }
 
-  for (int id = (executor_.get() == NULL ? 0 : 1); id < fragment_profiles_.size(); ++id) {
+  for (int id = (executor_ == NULL ? 0 : 1); id < fragment_profiles_.size(); ++id) {
     SummaryStats& acc = fragment_profiles_[id].bytes_assigned;
     double min = accumulators::min(acc);
     double max = accumulators::max(acc);
@@ -1360,6 +1229,7 @@ void Coordinator::PrintFragmentInstanceInfo() {
 
 void Coordinator::InitExecProfile(const TQueryExecRequest& request) {
   // Initialize the structure to collect execution summary of every plan node.
+  fragment_profiles_.resize(request.fragments.size());
   exec_summary_.__isset.nodes = true;
   for (int i = 0; i < request.fragments.size(); ++i) {
     if (!request.fragments[i].__isset.plan) continue;
@@ -1394,46 +1264,30 @@ void Coordinator::InitExecProfile(const TQueryExecRequest& request) {
     }
   }
 
-  if (executor_.get() != NULL) {
-    // register coordinator's fragment profile now, before those of the backends,
-    // so it shows up at the top
-    query_profile_->AddChild(executor_->profile());
-    executor_->profile()->set_name(Substitute("Coordinator Fragment $0",
-        request.fragments[0].display_name));
-    CollectScanNodeCounters(executor_->profile(), &coordinator_counters_);
-  }
-
   // Initialize the runtime profile structure. This adds the per fragment average
   // profiles followed by the per fragment instance profiles.
-  bool has_coordinator_fragment =
-      request.fragments[0].partition.type == TPartitionType::UNPARTITIONED;
-  fragment_profiles_.resize(request.fragments.size());
   for (int i = 0; i < request.fragments.size(); ++i) {
-    fragment_profiles_[i].num_instances = 0;
-
-    // Special case fragment idx 0 if there is a coordinator. There is only one
-    // instance of this profile so the average is just the coordinator profile.
-    if (i == 0 && has_coordinator_fragment) {
-      fragment_profiles_[i].averaged_profile = executor_->profile();
-      fragment_profiles_[i].num_instances = 1;
-      continue;
-    }
-    fragment_profiles_[i].averaged_profile =
-        obj_pool()->Add(new RuntimeProfile(obj_pool(),
-            Substitute("Averaged Fragment $0", request.fragments[i].display_name), true));
-    // Insert the avg profiles in ascending fragment number order. If
-    // there is a coordinator fragment, it's been placed in
-    // fragment_profiles_[0].averaged_profile, ensuring that this code
-    // will put the first averaged profile immediately after it. If
-    // there is no coordinator fragment, the first averaged profile
-    // will be inserted as the first child of query_profile_, and then
-    // all other averaged fragments will follow.
-    query_profile_->AddChild(fragment_profiles_[i].averaged_profile, true,
-        (i > 0) ? fragment_profiles_[i-1].averaged_profile : NULL);
-
+    // Insert the avg profiles in ascending fragment number order. If there is a
+    // coordinator fragment, it's been placed in fragment_profiles_[0].averaged_profile,
+    // ensuring that this code will put the first averaged profile immediately after
+    // it. If there is no coordinator fragment, the first averaged profile will be
+    // inserted as the first child of query_profile_, and then all other averaged
+    // fragments will follow.
+    bool is_coordinator_fragment = (i == 0 && schedule_.GetCoordFragment() != nullptr);
+    string profile_name =
+        Substitute(is_coordinator_fragment ? "Coordinator Fragment $0" : "Fragment $0",
+            request.fragments[i].display_name);
     fragment_profiles_[i].root_profile =
-        obj_pool()->Add(new RuntimeProfile(obj_pool(),
-            Substitute("Fragment $0", request.fragments[i].display_name)));
+        obj_pool()->Add(new RuntimeProfile(obj_pool(), profile_name));
+    if (is_coordinator_fragment) {
+      fragment_profiles_[i].averaged_profile = nullptr;
+    } else {
+      fragment_profiles_[i].averaged_profile = obj_pool()->Add(new RuntimeProfile(
+          obj_pool(),
+          Substitute("Averaged Fragment $0", request.fragments[i].display_name), true));
+      query_profile_->AddChild(fragment_profiles_[i].averaged_profile, true,
+          (i > 0) ? fragment_profiles_[i - 1].averaged_profile : NULL);
+    }
     // Note: we don't start the wall timer here for the fragment
     // profile; it's uninteresting and misleading.
     query_profile_->AddChild(fragment_profiles_[i].root_profile);
@@ -1490,40 +1344,23 @@ void Coordinator::MtInitExecProfiles() {
   schedule_.GetTPlanFragments(&fragments);
   fragment_profiles_.resize(fragments.size());
 
-  // start with coordinator fragment, if there is one
   const TPlanFragment* coord_fragment = schedule_.GetCoordFragment();
-  if (coord_fragment != NULL) {
-    DCHECK(executor_.get() != NULL);
-    PerFragmentProfileData* data = &fragment_profiles_[coord_fragment->idx];
-    data->num_instances = 1;
-    // TODO: fix this; this is not an averaged profile; we should follow the exact
-    // same structure we have for all other profiles (average + root + single
-    // instance profile)
-    data->averaged_profile = executor_->profile();
-
-    // register coordinator's fragment profile in the query profile now, before those
-    // of the fragment instances, so it shows up at the top
-    query_profile_->AddChild(executor_->profile());
-    executor_->profile()->set_name(Substitute("Coordinator Fragment $0",
-        coord_fragment->display_name));
-    CollectScanNodeCounters(executor_->profile(), &coordinator_counters_);
-  }
 
   // Initialize the runtime profile structure. This adds the per fragment average
   // profiles followed by the per fragment instance profiles.
   for (const TPlanFragment* fragment: fragments) {
-    if (fragment == coord_fragment) continue;
+    string profile_name =
+        (fragment == coord_fragment) ? "Coordinator Fragment $0" : "Fragment $0";
     PerFragmentProfileData* data = &fragment_profiles_[fragment->idx];
     data->num_instances =
         schedule_.GetFragmentExecParams(fragment->idx).instance_exec_params.size();
-
-    data->averaged_profile =
-        obj_pool()->Add(new RuntimeProfile(obj_pool(),
-          Substitute("Averaged Fragment $0", fragment->display_name), true));
-    query_profile_->AddChild(data->averaged_profile, true);
-    data->root_profile =
-        obj_pool()->Add(new RuntimeProfile(obj_pool(),
-          Substitute("Fragment $0", fragment->display_name)));
+    if (fragment != coord_fragment) {
+      data->averaged_profile = obj_pool()->Add(new RuntimeProfile(
+          obj_pool(), Substitute("Averaged Fragment $0", fragment->display_name), true));
+      query_profile_->AddChild(data->averaged_profile, true);
+    }
+    data->root_profile = obj_pool()->Add(
+        new RuntimeProfile(obj_pool(), Substitute(profile_name, fragment->display_name)));
     // Note: we don't start the wall timer here for the fragment profile;
     // it's uninteresting and misleading.
     query_profile_->AddChild(data->root_profile);
@@ -1637,8 +1474,6 @@ void Coordinator::ExecRemoteFragment(const FragmentExecParams& fragment_exec_par
             << " instance_id=" << PrintId(exec_state->fragment_instance_id())
             << " host=" << exec_state->impalad_address();
 
-  // Guard against concurrent UpdateExecStatus() that may arrive after RPC returns.
-  lock_guard<mutex> l(*exec_state->lock());
   int64_t start = MonotonicMillis();
 
   Status client_connect_status;
@@ -1685,32 +1520,27 @@ void Coordinator::Cancel(const Status* cause) {
   // if the query status indicates an error, cancellation has already been initiated
   if (!query_status_.ok()) return;
   // prevent others from cancelling a second time
+
+  // TODO: This should default to OK(), not CANCELLED if there is no cause (or callers
+  // should explicitly pass Status::OK()). Fragment instances may be cancelled at the end
+  // of a successful query. Need to clean up relationship between query_status_ here and
+  // in QueryExecState. See IMPALA-4279.
   query_status_ = (cause != NULL && !cause->ok()) ? *cause : Status::CANCELLED;
   CancelInternal();
 }
 
 void Coordinator::CancelInternal() {
   VLOG_QUERY << "Cancel() query_id=" << query_id_;
-  DCHECK(!query_status_.ok());
-
-  // cancel local fragment
-  if (executor_.get() != NULL) executor_->Cancel();
-
-  CancelRemoteFragments();
+  CancelFragmentInstances();
 
   // Report the summary with whatever progress the query made before being cancelled.
   ReportQuerySummary();
 }
 
-void Coordinator::CancelRemoteFragments() {
+void Coordinator::CancelFragmentInstances() {
+  int num_cancelled = 0;
   for (FragmentInstanceState* exec_state: fragment_instance_states_) {
     DCHECK(exec_state != nullptr);
-    if (exec_state->fragment_idx() == 0) continue;  // the coord fragment
-
-    // If a fragment failed before we finished issuing all remote fragments,
-    // this function will have been called before we finished populating
-    // fragment_instance_states_. Skip any such uninitialized exec states.
-    if (exec_state == NULL) continue;
 
     // lock each exec_state individually to synchronize correctly with
     // UpdateFragmentExecStatus() (which doesn't get the global lock_
@@ -1735,7 +1565,7 @@ void Coordinator::CancelRemoteFragments() {
     ImpalaBackendConnection backend_client(
         exec_env_->impalad_client_cache(), exec_state->impalad_address(), &status);
     if (!status.ok()) continue;
-
+    ++num_cancelled;
     TCancelPlanFragmentParams params;
     params.protocol_version = ImpalaInternalServiceVersion::V1;
     params.__set_fragment_instance_id(exec_state->fragment_instance_id());
@@ -1765,8 +1595,11 @@ void Coordinator::CancelRemoteFragments() {
       exec_state->status()->AddDetail(join(res.status.error_msgs, "; "));
     }
   }
+  VLOG_QUERY << Substitute(
+      "CancelFragmentInstances() query_id=$0, tried to cancel $1 fragment instances",
+      PrintId(query_id_), num_cancelled);
 
-  // notify that we completed with an error
+  // Notify that we completed with an error.
   instance_completion_cv_.notify_all();
 }
 
@@ -1799,11 +1632,11 @@ Status Coordinator::UpdateFragmentExecStatus(const TReportExecStatusParams& para
       // We can't update this backend's profile if ReportQuerySummary() is running,
       // because it depends on all profiles not changing during its execution (when it
       // calls SortChildren()). ReportQuerySummary() only gets called after
-      // WaitForAllInstances() returns or at the end of CancelRemoteFragments().
+      // WaitForAllInstances() returns or at the end of CancelFragmentInstances().
       // WaitForAllInstances() only returns after all backends have completed (in which
       // case we wouldn't be in this function), or when there's an error, in which case
-      // CancelRemoteFragments() is called. CancelRemoteFragments sets all exec_state's
-      // statuses to cancelled.
+      // CancelFragmentInstances() is called. CancelFragmentInstances sets all
+      // exec_state's statuses to cancelled.
       // TODO: We're losing this profile information. Call ReportQuerySummary only after
       // all backends have completed.
       exec_state->profile()->Update(cumulative_profile);
@@ -1899,18 +1732,12 @@ Status Coordinator::UpdateFragmentExecStatus(const TReportExecStatusParams& para
   return Status::OK();
 }
 
-const RowDescriptor& Coordinator::row_desc() const {
-  DCHECK(executor_.get() != NULL);
-  return executor_->row_desc();
-}
-
 RuntimeState* Coordinator::runtime_state() {
-  return executor_.get() == NULL ? NULL : executor_->runtime_state();
+  return executor_ == NULL ? NULL : executor_->runtime_state();
 }
 
 MemTracker* Coordinator::query_mem_tracker() {
-  return executor_.get() == NULL ? query_mem_tracker_.get() :
-      executor_->runtime_state()->query_mem_tracker();
+  return query_mem_tracker_.get();
 }
 
 bool Coordinator::PrepareCatalogUpdate(TUpdateCatalogRequest* catalog_update) {
@@ -1941,7 +1768,9 @@ void Coordinator::UpdateAverageProfile(FragmentInstanceState* instance_state) {
   PerFragmentProfileData* data = &fragment_profiles_[fragment_idx];
 
   // No locks are taken since UpdateAverage() and AddChild() take their own locks
-  data->averaged_profile->UpdateAverage(instance_state->profile());
+  if (data->averaged_profile != nullptr) {
+    data->averaged_profile->UpdateAverage(instance_state->profile());
+  }
   data->root_profile->AddChild(instance_state->profile());
 }
 
@@ -2000,18 +1829,18 @@ void Coordinator::UpdateExecSummary(const FragmentInstanceState& instance_state)
 
 // This function appends summary information to the query_profile_ before
 // outputting it to VLOG.  It adds:
-//   1. Averaged remote fragment profiles (TODO: add outliers)
-//   2. Summary of remote fragment durations (min, max, mean, stddev)
-//   3. Summary of remote fragment rates (min, max, mean, stddev)
+//   1. Averaged fragment instance profiles (TODO: add outliers)
+//   2. Summary of fragment instance durations (min, max, mean, stddev)
+//   3. Summary of fragment instance rates (min, max, mean, stddev)
 // TODO: add histogram/percentile
 void Coordinator::ReportQuerySummary() {
-  // In this case, the query did not even get to start on all the remote nodes,
-  // some of the state that is used below might be uninitialized.  In this case,
+  // In this case, the query did not even get to start all fragment instances.
+  // Some of the state that is used below might be uninitialized.  In this case,
   // the query has made so little progress, reporting a summary is not very useful.
   if (!has_called_wait_) return;
 
   if (!fragment_instance_states_.empty()) {
-    // Average all remote fragments for each fragment.
+    // Average all fragment instances for each fragment.
     for (FragmentInstanceState* state: fragment_instance_states_) {
       // TODO: make profiles uniform across all fragments so we don't have
       // to keep special-casing the coord fragment
@@ -2028,7 +1857,7 @@ void Coordinator::ReportQuerySummary() {
 
     InstanceComparator comparator;
     // Per fragment instances have been collected, output summaries
-    for (int i = (executor_.get() != NULL ? 1 : 0); i < fragment_profiles_.size(); ++i) {
+    for (int i = (executor_ != NULL ? 1 : 0); i < fragment_profiles_.size(); ++i) {
       fragment_profiles_[i].root_profile->SortChildren(comparator);
       SummaryStats& completion_times = fragment_profiles_[i].completion_times;
       SummaryStats& rates = fragment_profiles_[i].rates;
@@ -2088,16 +1917,6 @@ void Coordinator::ReportQuerySummary() {
 
 string Coordinator::GetErrorLog() {
   ErrorLogMap merged;
-  {
-    lock_guard<mutex> l(lock_);
-    // TODO-MT: use FragmentInstanceState::error_log_ instead
-    // as part of getting rid of the special-casing of the coordinator instance
-    if (executor_.get() != NULL && executor_->runtime_state() != NULL) {
-      ErrorLogMap runtime_error_log;
-      executor_->runtime_state()->GetErrors(&runtime_error_log);
-      MergeErrorMaps(&merged, runtime_error_log);
-    }
-  }
   for (FragmentInstanceState* state: fragment_instance_states_) {
     lock_guard<mutex> l(*state->lock());
     if (state->error_log()->size() > 0)  MergeErrorMaps(&merged, *state->error_log());
@@ -2285,11 +2104,20 @@ void DistributeFilters(shared_ptr<TPublishFilterParams> params,
 void Coordinator::TearDown() {
   DCHECK(!torn_down_) << "Coordinator::TearDown() may not be called twice";
   torn_down_ = true;
-  lock_guard<SpinLock> l(filter_lock_);
-  for (auto& filter: filter_routing_table_) {
-    FilterState* state = &filter.second;
-    state->Disable(filter_mem_tracker_.get());
+  if (filter_routing_table_.size() > 0) {
+    query_profile_->AddInfoString("Final filter table", FilterDebugString());
   }
+
+  {
+    lock_guard<SpinLock> l(filter_lock_);
+    for (auto& filter : filter_routing_table_) {
+      FilterState* state = &filter.second;
+      state->Disable(filter_mem_tracker_.get());
+    }
+  }
+
+  // Need to protect against failed Prepare(), where root_sink() would not be set.
+  if (root_sink_ != nullptr) root_sink_->CloseConsumer();
 }
 
 void Coordinator::UpdateFilter(const TUpdateFilterParams& params) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/runtime/coordinator.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/coordinator.h b/be/src/runtime/coordinator.h
index bb67377..f73cf42 100644
--- a/be/src/runtime/coordinator.h
+++ b/be/src/runtime/coordinator.h
@@ -34,16 +34,18 @@
 #include <boost/thread/mutex.hpp>
 #include <boost/thread/condition_variable.hpp>
 
+#include "common/global-types.h"
 #include "common/hdfs.h"
 #include "common/status.h"
-#include "common/global-types.h"
-#include "util/progress-updater.h"
-#include "util/histogram-metric.h"
-#include "util/runtime-profile.h"
+#include "gen-cpp/Frontend_types.h"
+#include "gen-cpp/Types_types.h"
 #include "runtime/runtime-state.h"
 #include "scheduling/simple-scheduler.h"
-#include "gen-cpp/Types_types.h"
-#include "gen-cpp/Frontend_types.h"
+#include "service/fragment-exec-state.h"
+#include "service/fragment-mgr.h"
+#include "util/histogram-metric.h"
+#include "util/progress-updater.h"
+#include "util/runtime-profile.h"
 
 namespace impala {
 
@@ -67,28 +69,33 @@ class TRuntimeProfileTree;
 class RuntimeProfile;
 class TablePrinter;
 class TPlanFragment;
+class QueryResultSet;
 
 struct DebugOptions;
 
-/// Query coordinator: handles execution of plan fragments on remote nodes, given
-/// a TQueryExecRequest. As part of that, it handles all interactions with the
-/// executing backends; it is also responsible for implementing all client requests
-/// regarding the query, including cancellation.
-/// The coordinator fragment is executed locally in the calling thread, all other
-/// fragments are sent to remote nodes. The coordinator also monitors
-/// the execution status of the remote fragments and aborts the entire query if an error
-/// occurs, either in any of the remote fragments or in the local fragment.
+/// Query coordinator: handles execution of fragment instances on remote nodes, given a
+/// TQueryExecRequest. As part of that, it handles all interactions with the executing
+/// backends; it is also responsible for implementing all client requests regarding the
+/// query, including cancellation.
+///
+/// The coordinator monitors the execution status of fragment instances and aborts the
+/// entire query if an error is reported by any of them.
+///
+/// Queries that have results have those results fetched by calling GetNext(). Results
+/// rows are produced by a fragment instance that always executes on the same machine as
+/// the coordinator.
+///
 /// Once a query has finished executing and all results have been returned either to the
 /// caller of GetNext() or a data sink, execution_completed() will return true. If the
-/// query is aborted, execution_completed should also be set to true.
-/// Coordinator is thread-safe, with the exception of GetNext().
+/// query is aborted, execution_completed should also be set to true. Coordinator is
+/// thread-safe, with the exception of GetNext().
 //
 /// A typical sequence of calls for a single query (calls under the same numbered
 /// item can happen concurrently):
 /// 1. client: Exec()
 /// 2. client: Wait()/client: Cancel()/backend: UpdateFragmentExecStatus()
 /// 3. client: GetNext()*/client: Cancel()/backend: UpdateFragmentExecStatus()
-//
+///
 /// The implementation ensures that setting an overall error status and initiating
 /// cancellation of local and all remote fragments is atomic.
 ///
@@ -104,14 +111,10 @@ class Coordinator {
       RuntimeProfile::EventSequence* events);
   ~Coordinator();
 
-  /// Initiate asynchronous execution of a query with the given schedule. Returns as soon
-  /// as all plan fragments have started executing at their respective backends.
-  /// 'schedule' must contain at least a coordinator plan fragment (ie, can't
-  /// be for a query like 'SELECT 1').
-  /// Populates and prepares output_expr_ctxs from the coordinator's fragment if there is
-  /// one, and LLVM optimizes them together with the fragment's other exprs.
+  /// Initiate asynchronous execution of a query with the given schedule. When it returns,
+  /// all fragment instances have started executing at their respective backends.
   /// A call to Exec() must precede all other member function calls.
-  Status Exec(std::vector<ExprContext*>* output_expr_ctxs);
+  Status Exec();
 
   /// Blocks until result rows are ready to be retrieved via GetNext(), or, if the
   /// query doesn't return rows, until the query finishes or is cancelled.
@@ -120,25 +123,25 @@ class Coordinator {
   /// Wait() calls concurrently.
   Status Wait();
 
-  /// Returns tuples from the coordinator fragment. Any returned tuples are valid until
-  /// the next GetNext() call. If *batch is NULL, execution has completed and GetNext()
-  /// must not be called again.
-  /// GetNext() will not set *batch=NULL until all fragment instances have
-  /// either completed or have failed.
-  /// It is safe to call GetNext() even in the case where there is no coordinator fragment
-  /// (distributed INSERT).
-  /// '*batch' is owned by the underlying PlanFragmentExecutor and must not be deleted.
-  /// *state is owned by the caller, and must not be deleted.
-  /// Returns an error status if an error was encountered either locally or by
-  /// any of the remote fragments or if the query was cancelled.
-  /// GetNext() is not thread-safe: multiple threads must not make concurrent
-  /// GetNext() calls (but may call any of the other member functions concurrently
-  /// with GetNext()).
-  Status GetNext(RowBatch** batch, RuntimeState* state);
+  /// Fills 'results' with up to 'max_rows' rows. May return fewer than 'max_rows'
+  /// rows, but will not return more.
+  ///
+  /// If *eos is true, execution has completed and GetNext() must not be called
+  /// again.
+  ///
+  /// GetNext() will not set *eos=true until all fragment instances have either completed
+  /// or have failed.
+  ///
+  /// Returns an error status if an error was encountered either locally or by any of the
+  /// remote fragments or if the query was cancelled.
+  ///
+  /// GetNext() is not thread-safe: multiple threads must not make concurrent GetNext()
+  /// calls (but may call any of the other member functions concurrently with GetNext()).
+  Status GetNext(QueryResultSet* results, int max_rows, bool* eos);
 
   /// Cancel execution of query. This includes the execution of the local plan fragment,
-  /// if any, as well as all plan fragments on remote nodes. Sets query_status_ to
-  /// the given cause if non-NULL. Otherwise, sets query_status_ to Status::CANCELLED.
+  /// if any, as well as all plan fragments on remote nodes. Sets query_status_ to the
+  /// given cause if non-NULL. Otherwise, sets query_status_ to Status::CANCELLED.
   /// Idempotent.
   void Cancel(const Status* cause = NULL);
 
@@ -151,12 +154,18 @@ class Coordinator {
   /// to CancelInternal().
   Status UpdateFragmentExecStatus(const TReportExecStatusParams& params);
 
-  /// only valid *after* calling Exec(), and may return NULL if there is no executor
+  /// Only valid *after* calling Exec(). Return nullptr if the running query does not
+  /// produce any rows.
+  ///
+  /// TODO: The only dependency on this is QueryExecState, used to track memory for the
+  /// result cache. Remove this dependency, possibly by moving result caching inside this
+  /// class.
   RuntimeState* runtime_state();
-  const RowDescriptor& row_desc() const;
 
   /// Only valid after Exec(). Returns runtime_state()->query_mem_tracker() if there
   /// is a coordinator fragment, or query_mem_tracker_ (initialized in Exec()) otherwise.
+  ///
+  /// TODO: Remove, see runtime_state().
   MemTracker* query_mem_tracker();
 
   /// Get cumulative profile aggregated over all fragments of the query.
@@ -275,8 +284,18 @@ class Coordinator {
   /// Once this is set to true, errors from remote fragments are ignored.
   bool returned_all_results_;
 
-  /// execution state of coordinator fragment
-  boost::scoped_ptr<PlanFragmentExecutor> executor_;
+  /// Non-null if and only if the query produces results for the client; i.e. is of
+  /// TStmtType::QUERY. Coordinator uses these to pull results from plan tree and return
+  /// them to the client in GetNext(), and also to access the fragment instance's runtime
+  /// state.
+  ///
+  /// Result rows are materialized by this fragment instance in its own thread. They are
+  /// materialized into a QueryResultSet provided to the coordinator during GetNext().
+  ///
+  /// Not owned by this class, created during fragment instance start-up by
+  /// FragmentExecState and set here in Exec().
+  PlanFragmentExecutor* executor_ = nullptr;
+  PlanRootSink* root_sink_ = nullptr;
 
   /// Query mem tracker for this coordinator initialized in Exec(). Only valid if there
   /// is no coordinator fragment (i.e. executor_ == NULL). If executor_ is not NULL,
@@ -383,7 +402,7 @@ class Coordinator {
   RuntimeProfile::Counter* finalization_timer_;
 
   /// Barrier that is released when all calls to ExecRemoteFragment() have
-  /// returned, successfully or not. Initialised during StartRemoteFragments().
+  /// returned, successfully or not. Initialised during Exec().
   boost::scoped_ptr<CountingBarrier> exec_complete_barrier_;
 
   /// Represents a runtime filter target.
@@ -465,10 +484,9 @@ class Coordinator {
   /// Runs cancel logic. Assumes that lock_ is held.
   void CancelInternal();
 
-  /// Cancels remote fragments. Assumes that lock_ is held.  This can be called when
-  /// the query is not being cancelled in the case where the query limit is
-  /// reached.
-  void CancelRemoteFragments();
+  /// Cancels all fragment instances. Assumes that lock_ is held. This may be called when
+  /// the query is not being cancelled in the case where the query limit is reached.
+  void CancelFragmentInstances();
 
   /// Acquires lock_ and updates query_status_ with 'status' if it's not already
   /// an error status, and returns the current query_status_.
@@ -531,30 +549,18 @@ class Coordinator {
   void PopulatePathPermissionCache(hdfsFS fs, const std::string& path_str,
       PermissionCache* permissions_cache);
 
-  /// Validates that all collection-typed slots in the given batch are set to NULL.
-  /// See SubplanNode for details on when collection-typed slots are set to NULL.
-  /// TODO: This validation will become obsolete when we can return collection values.
-  /// We will then need a different mechanism to assert the correct behavior of the
-  /// SubplanNode with respect to setting collection-slots to NULL.
-  void ValidateCollectionSlots(RowBatch* batch);
-
-  /// Prepare coordinator fragment for execution (update filter routing table,
-  /// prepare executor, set up output exprs) and create its FragmentInstanceState.
-  Status PrepareCoordFragment(std::vector<ExprContext*>* output_expr_ctxs);
-
-  /// Starts all remote fragments contained in the schedule by issuing RPCs in parallel,
+  /// Starts all fragment instances contained in the schedule by issuing RPCs in parallel,
   /// and then waiting for all of the RPCs to complete.
-  void StartRemoteFragments();
+  void StartFragments();
 
-  /// Starts all remote fragment instances contained in the schedule by issuing RPCs in
-  /// parallel and then waiting for all of the RPCs to complete. Also sets up and
-  /// registers the state for all non-coordinator fragment instance.
-  void MtStartRemoteFInstances();
+  /// Starts all fragment instances contained in the schedule by issuing RPCs in parallel
+  /// and then waiting for all of the RPCs to complete.
+  void MtStartFInstances();
 
   /// Calls CancelInternal() and returns an error if there was any error starting the
   /// fragments.
   /// Also updates query_profile_ with the startup latency histogram.
-  Status FinishRemoteInstanceStartup();
+  Status FinishInstanceStartup();
 
   /// Build the filter routing table by iterating over all plan nodes and collecting the
   /// filters that they either produce or consume. The source and target fragment

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/runtime/exec-env.cc
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.cc b/be/src/runtime/exec-env.cc
index 1b3fa14..db0d242 100644
--- a/be/src/runtime/exec-env.cc
+++ b/be/src/runtime/exec-env.cc
@@ -24,6 +24,8 @@
 #include <gutil/strings/substitute.h>
 
 #include "common/logging.h"
+#include "gen-cpp/CatalogService.h"
+#include "gen-cpp/ImpalaInternalService.h"
 #include "runtime/backend-client.h"
 #include "runtime/client-cache.h"
 #include "runtime/coordinator.h"
@@ -36,25 +38,24 @@
 #include "runtime/thread-resource-mgr.h"
 #include "runtime/tmp-file-mgr.h"
 #include "scheduling/request-pool-service.h"
-#include "service/frontend.h"
 #include "scheduling/simple-scheduler.h"
+#include "service/fragment-mgr.h"
+#include "service/frontend.h"
 #include "statestore/statestore-subscriber.h"
 #include "util/debug-util.h"
+#include "util/debug-util.h"
 #include "util/default-path-handlers.h"
 #include "util/hdfs-bulk-ops.h"
 #include "util/mem-info.h"
+#include "util/mem-info.h"
+#include "util/memory-metrics.h"
+#include "util/memory-metrics.h"
 #include "util/metrics.h"
 #include "util/network-util.h"
 #include "util/parse-util.h"
-#include "util/memory-metrics.h"
-#include "util/webserver.h"
-#include "util/mem-info.h"
-#include "util/debug-util.h"
-#include "util/memory-metrics.h"
 #include "util/pretty-printer.h"
 #include "util/thread-pool.h"
-#include "gen-cpp/ImpalaInternalService.h"
-#include "gen-cpp/CatalogService.h"
+#include "util/webserver.h"
 
 #include "common/names.h"
 
@@ -145,6 +146,7 @@ ExecEnv::ExecEnv()
     fragment_exec_thread_pool_(new CallableThreadPool("coordinator-fragment-rpc",
         "worker", FLAGS_coordinator_rpc_threads, numeric_limits<int32_t>::max())),
     async_rpc_pool_(new CallableThreadPool("rpc-pool", "async-rpc-sender", 8, 10000)),
+    fragment_mgr_(new FragmentMgr()),
     enable_webserver_(FLAGS_enable_webserver),
     is_fe_tests_(false),
     backend_address_(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port)) {
@@ -197,6 +199,7 @@ ExecEnv::ExecEnv(const string& hostname, int backend_port, int subscriber_port,
     fragment_exec_thread_pool_(new CallableThreadPool("coordinator-fragment-rpc",
         "worker", FLAGS_coordinator_rpc_threads, numeric_limits<int32_t>::max())),
     async_rpc_pool_(new CallableThreadPool("rpc-pool", "async-rpc-sender", 8, 10000)),
+    fragment_mgr_(new FragmentMgr()),
     enable_webserver_(FLAGS_enable_webserver && webserver_port > 0),
     is_fe_tests_(false),
     backend_address_(MakeNetworkAddress(FLAGS_hostname, FLAGS_be_port)) {

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/be/src/runtime/exec-env.h
----------------------------------------------------------------------
diff --git a/be/src/runtime/exec-env.h b/be/src/runtime/exec-env.h
index 303876f..718c6d0 100644
--- a/be/src/runtime/exec-env.h
+++ b/be/src/runtime/exec-env.h
@@ -96,6 +96,7 @@ class ExecEnv {
   Frontend* frontend() { return frontend_.get(); };
   RequestPoolService* request_pool_service() { return request_pool_service_.get(); }
   CallableThreadPool* rpc_pool() { return async_rpc_pool_.get(); }
+  FragmentMgr* fragment_mgr() { return fragment_mgr_.get(); }
 
   void set_enable_webserver(bool enable) { enable_webserver_ = enable; }
 
@@ -137,6 +138,7 @@ class ExecEnv {
   boost::scoped_ptr<Frontend> frontend_;
   boost::scoped_ptr<CallableThreadPool> fragment_exec_thread_pool_;
   boost::scoped_ptr<CallableThreadPool> async_rpc_pool_;
+  boost::scoped_ptr<FragmentMgr> fragment_mgr_;
 
   /// Not owned by this class
   ImpalaServer* impala_server_;


[3/7] incubator-impala git commit: IMPALA-2905: Handle coordinator fragment lifecycle like all others

Posted by he...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/order.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/order.test b/testdata/workloads/functional-planner/queries/PlannerTest/order.test
index a39b9b2..266c517 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/order.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/order.test
@@ -2,12 +2,16 @@ select name, zip
 from functional.testtbl
 order by name offset 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SORT
 |  order by: name ASC
 |
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  offset: 5
 |  order by: name ASC
@@ -22,12 +26,16 @@ select name, zip
 from functional.testtbl
 order by name
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SORT
 |  order by: name ASC
 |
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: name ASC
 |
@@ -43,6 +51,8 @@ where name like 'm%'
 group by 1
 order by 2 desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: count(*) DESC
 |
@@ -54,6 +64,8 @@ order by 2 desc
    partitions=1/1 files=0 size=0B
    predicates: name LIKE 'm%'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC
 |
@@ -80,6 +92,8 @@ where id < 5
 group by 1
 order by 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: sum(float_col) ASC
 |
@@ -90,6 +104,8 @@ order by 2
 00:SCAN HBASE [functional_hbase.alltypessmall]
    predicates: id < 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(float_col) ASC
 |
@@ -114,6 +130,8 @@ from functional_hbase.alltypessmall
 group by 1
 order by 2,3 desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: sum(float_col) ASC, min(float_col) DESC
 |
@@ -123,6 +141,8 @@ order by 2,3 desc
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(float_col) ASC, min(float_col) DESC
 |
@@ -144,12 +164,16 @@ order by 2,3 desc
 # Test that the sort is on int_col and not on the id column
 select int_col as id from functional.alltypessmall order by id
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SORT
 |  order by: int_col ASC
 |
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: int_col ASC
 |
@@ -162,12 +186,16 @@ select int_col as id from functional.alltypessmall order by id
 # Test that the sort is on id and not on int_col
 select int_col as id from functional.alltypessmall order by functional.alltypessmall.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SORT
 |  order by: id ASC
 |
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: id ASC
 |
@@ -184,6 +212,8 @@ select int_col, bigint_col from
    select * from functional.alltypessmall) t
 order by int_col desc offset 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SORT
 |  order by: int_col DESC
 |
@@ -195,6 +225,8 @@ order by int_col desc offset 5
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:MERGING-EXCHANGE [UNPARTITIONED]
 |  offset: 5
 |  order by: int_col DESC
@@ -217,6 +249,8 @@ select int_col, bigint_col from
    select * from functional.alltypessmall) t
 order by int_col desc offset 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: int_col DESC
 |
@@ -231,6 +265,8 @@ order by int_col desc offset 5
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:MERGING-EXCHANGE [UNPARTITIONED]
 |  offset: 5
 |  order by: int_col DESC
@@ -259,6 +295,8 @@ select j.*, d.* from functional.JoinTbl j full outer join functional.DimTbl d
 on (j.test_id = d.id)
 order by j.test_id, j.test_name, j.test_zip, j.alltypes_id, d.name
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SORT
 |  order by: test_id ASC, test_name ASC, test_zip ASC, alltypes_id ASC, name ASC
 |
@@ -271,6 +309,8 @@ order by j.test_id, j.test_name, j.test_zip, j.alltypes_id, d.name
 00:SCAN HDFS [functional.jointbl j]
    partitions=1/1 files=1 size=433B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: test_id ASC, test_name ASC, test_zip ASC, alltypes_id ASC, name ASC
 |
@@ -303,6 +343,8 @@ and c.string_col < '7'
 and a.int_col + b.float_col + cast(c.string_col as float) < 1000
 order by c.string_col desc, a.smallint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:SORT
 |  order by: string_col DESC, smallint_col ASC
 |
@@ -329,6 +371,8 @@ order by c.string_col desc, a.smallint_col
    predicates: c.string_col < '7'
    runtime filters: RF000 -> c.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: string_col DESC, smallint_col ASC
 |
@@ -370,6 +414,8 @@ from functional.alltypesagg
 group by 1
 order by avg(tinyint_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: avg(tinyint_col) ASC
 |
@@ -380,6 +426,8 @@ order by avg(tinyint_col)
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: avg(tinyint_col) ASC
 |
@@ -406,6 +454,8 @@ left outer join functional.alltypessmall t2
   on (t1.int_col = t2.int_col)
 order by t1.id,t2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SORT
 |  order by: id ASC, id ASC
 |
@@ -418,6 +468,8 @@ order by t1.id,t2.id
 00:SCAN HDFS [functional.alltypesagg t1]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: id ASC, id ASC
 |
@@ -440,6 +492,8 @@ select t1.id, t2.id from functional.alltypestiny t1 cross join functional.alltyp
 where (t1.id < 3 and t2.id < 3)
 order by t1.id, t2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SORT
 |  order by: id ASC, id ASC
 |
@@ -453,6 +507,8 @@ order by t1.id, t2.id
    partitions=4/4 files=4 size=460B
    predicates: t1.id < 3
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: id ASC, id ASC
 |
@@ -481,6 +537,8 @@ union distinct
 (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from functional.alltypestiny where year=2009 and month=2)
 order by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:SORT
 |  order by: id ASC
 |
@@ -501,6 +559,8 @@ order by 1
 01:SCAN HDFS [functional.alltypestiny]
    partitions=1/4 files=1 size=115B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: id ASC
 |
@@ -538,6 +598,8 @@ union all
 (select id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month from functional.alltypestiny where year=2009 and month=2)
 order by 1,2
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: id ASC, bool_col ASC
 |
@@ -560,6 +622,8 @@ order by 1,2
 05:SCAN HDFS [functional.alltypestiny]
    partitions=1/4 files=1 size=115B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: id ASC, bool_col ASC
 |
@@ -611,6 +675,8 @@ union all
    order by 1 limit 3)
 order by 12, 13, 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 16:SORT
 |  order by: year ASC, month ASC, id ASC
 |
@@ -658,6 +724,8 @@ order by 12, 13, 1
 10:SCAN HDFS [functional.alltypestiny]
    partitions=1/4 files=1 size=115B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: year ASC, month ASC, id ASC
 |
@@ -729,6 +797,8 @@ order by 12, 13, 1
 select * from (select * from functional.alltypes order by bigint_col limit 10) t
 order by int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: int_col ASC
 |
@@ -738,6 +808,8 @@ order by int_col
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: int_col ASC
 |
@@ -758,6 +830,8 @@ select * from
    (select * from functional.alltypessmall) order by bigint_col limit 10) t
 order by int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: int_col ASC
 |
@@ -772,6 +846,8 @@ order by int_col
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: int_col ASC
 |
@@ -796,6 +872,8 @@ select * from
 (select * from functional.alltypes order by bigint_col) A
 join B on (A.string_col = B.string_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypes.string_col = functional.alltypes.string_col
 |  runtime filters: RF000 <- functional.alltypes.string_col
@@ -807,6 +885,8 @@ join B on (A.string_col = B.string_col)
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> functional.alltypes.string_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -863,6 +943,8 @@ select * from functional.alltypes
    union all
 select * from functional.alltypessmall order by bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HDFS [functional.alltypessmall]
@@ -871,6 +953,8 @@ select * from functional.alltypessmall order by bigint_col
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -886,6 +970,8 @@ select * from functional.alltypes
    union all
 (select * from functional.alltypessmall) order by bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SORT
 |  order by: bigint_col ASC
 |
@@ -897,6 +983,8 @@ select * from functional.alltypes
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: bigint_col ASC
 |
@@ -917,6 +1005,8 @@ select int_col from functional.alltypes order by int_col
  union (select int_col from functional.alltypes order by int_col limit 10 offset 5)
 order by int_col offset 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: int_col ASC
 |
@@ -940,6 +1030,8 @@ order by int_col offset 5
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  offset: 5
 |  order by: int_col ASC
@@ -1003,6 +1095,8 @@ select * from
   having sum(float_col) > 10) t3
 order by x
 ---- PLAN
+PLAN-ROOT SINK
+|
 11:SORT
 |  order by: x ASC
 |
@@ -1040,6 +1134,8 @@ order by x
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:SORT
 |  order by: x ASC
 |
@@ -1098,6 +1194,8 @@ select int_col from
   (select int_col, bigint_col from functional.alltypesagg)
 order by bigint_col limit 10) A
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=10]
 |  order by: bigint_col ASC
 |
@@ -1109,6 +1207,8 @@ order by bigint_col limit 10) A
 01:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: bigint_col ASC
 |  limit: 10
@@ -1127,6 +1227,8 @@ order by bigint_col limit 10) A
 # Sort node is unnecessary (IMPALA-1148).
 select 1 from functional.alltypes order by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ====
@@ -1135,6 +1237,8 @@ select a from
   (select 1 as a, int_col, bigint_col
    from functional.alltypes order by 1 limit 1) v
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
    limit: 1

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/outer-joins.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/outer-joins.test b/testdata/workloads/functional-planner/queries/PlannerTest/outer-joins.test
index c6ccc81..f3e43bd 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/outer-joins.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/outer-joins.test
@@ -27,6 +27,8 @@ and t1.zip + t2.zip = 10
 # join predicate between t1, t2 and t3 applied after last join
 and t1.zip + t2.zip + t3.zip= 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: t1.id = t3.id
 |  other join predicates: t2.id = 15, t1.id - t2.id = 0
@@ -49,6 +51,8 @@ and t1.zip + t2.zip + t3.zip= 20
    partitions=1/1 files=0 size=0B
    predicates: t1.id > 0
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
@@ -87,6 +91,8 @@ from (select * from functional.testtbl a1) t1
 where t1.id > 0 and t2.id is null and t3.id is not null
 and t1.zip + t2.zip = 10 and t1.zip + t2.zip + t3.zip= 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a1.id = a3.id
 |  other join predicates: a2.id = 15, a1.id - a2.id = 0
@@ -109,6 +115,8 @@ and t1.zip + t2.zip = 10 and t1.zip + t2.zip + t3.zip= 20
    partitions=1/1 files=0 size=0B
    predicates: a1.id > 0
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
@@ -145,6 +153,8 @@ from functional.testtbl t1
     t1.id = t2.id and t1.id = 17)
   join functional.testtbl t3 on (t1.id = t3.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t3.id
 |  runtime filters: RF000 <- t3.id
@@ -195,6 +205,8 @@ and t1.zip + t2.zip = 10
 # join predicate between t1, t2 and t3 applied after last join
 and t1.zip + t2.zip + t3.zip= 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: t1.id = t3.id
 |  other join predicates: t3.zip = 94720, t1.id - t2.id = 0
@@ -219,6 +231,8 @@ and t1.zip + t2.zip + t3.zip= 20
    predicates: t1.id IS NOT NULL, t1.id > 0
    runtime filters: RF000 -> t1.id, RF001 -> t1.id - 1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -262,6 +276,8 @@ from (select * from functional.testtbl a1) t1 right outer join (select * from fu
 where t1.id > 0 and t2.id is null and t3.id is not null
 and t1.zip + t2.zip = 10 and t1.zip + t2.zip + t3.zip= 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: a1.id = a3.id
 |  other join predicates: a3.zip = 94720, a1.id - a2.id = 0
@@ -286,6 +302,8 @@ and t1.zip + t2.zip = 10 and t1.zip + t2.zip + t3.zip= 20
    predicates: a1.id IS NOT NULL, a1.id > 0
    runtime filters: RF000 -> a1.id, RF001 -> a1.id - 1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -327,6 +345,8 @@ from functional.alltypesagg a
 right outer join functional.alltypestiny b on (a.tinyint_col = b.id)
 where a.tinyint_col is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: a.tinyint_col = b.id
 |  other predicates: a.tinyint_col IS NULL
@@ -339,6 +359,8 @@ where a.tinyint_col is null
    partitions=11/11 files=11 size=814.73KB
    runtime filters: RF000 -> a.tinyint_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -363,6 +385,8 @@ from functional.alltypesagg a
 full outer join functional.alltypestiny b on (a.tinyint_col = b.id)
 where a.tinyint_col is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: a.tinyint_col = b.id
 |  other predicates: a.tinyint_col IS NULL
@@ -373,6 +397,8 @@ where a.tinyint_col is null
 00:SCAN HDFS [functional.alltypesagg a]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
@@ -398,6 +424,8 @@ from functional.alltypes a full outer join functional.alltypes b
 # also to the full outer join
 where b.bigint_col > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: a.id = b.id
 |  other join predicates: a.int_col < 10, b.tinyint_col != 5
@@ -420,6 +448,8 @@ inner join functional.alltypes c
 # first full outer join
 where b.tinyint_col > 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [INNER JOIN]
 |  hash predicates: c.int_col = a.int_col
 |  other predicates: a.tinyint_col < 10
@@ -453,6 +483,8 @@ full outer join functional.alltypes c
 # re-assigned to the full outer join.
 where a.smallint_col = 100 and a.float_col > b.float_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: c.int_col = a.int_col
 |  other join predicates: a.bigint_col < 10, a.tinyint_col < b.tinyint_col
@@ -487,6 +519,8 @@ full outer join functional.alltypes d
 # predicate on b from the where clause is assigned to the first full outer join
 where a.bool_col = false and a.float_col < b.float_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: a.tinyint_col = d.tinyint_col
 |  other join predicates: b.int_col < 20
@@ -536,6 +570,8 @@ from (
 # tuple ids
 where x != y
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: c.id = d.id
 |  other join predicates: a.bigint_col > b.bigint_col
@@ -575,6 +611,8 @@ full outer join functional.alltypes c
 # that materializes the corresponding tuple ids
 where a.bigint_col = b.bigint_col and a.tinyint_col < b.tinyint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: b.int_col = c.int_col
 |  other join predicates: c.int_col < 10
@@ -606,6 +644,8 @@ full outer join functional.alltypes d
   on (b.string_col = d.string_col and a.tinyint_col < b.tinyint_col)
 where a.float_col = b.float_col and b.smallint_col = 1 and d.tinyint_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: b.string_col = d.string_col
 |  other join predicates: a.tinyint_col < b.tinyint_col
@@ -645,6 +685,8 @@ group by a.bool_col, a.int_col, b.bool_col, b.int_col
 having a.bool_col is null and a.int_col is not null
   and b.bool_col is null and b.int_col is not null
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(b.double_col)
 |  group by: a.bool_col, a.int_col, b.bool_col, b.int_col
@@ -669,6 +711,8 @@ group by a.bool_col, a.int_col, b.bool_col, b.int_col
 having a.bool_col is null and a.int_col is not null
   and b.bool_col is null and b.int_col is not null
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(b.double_col)
 |  group by: a.bool_col, a.int_col, b.bool_col, b.int_col
@@ -697,6 +741,8 @@ where (
     a.timestamp_col
   end) >= cast('2001-01-01 00:00:00' as timestamp);
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: b.id = a.id
 |  runtime filters: RF000 <- a.id
@@ -717,6 +763,8 @@ left outer join functional.alltypestiny b
 inner join functional.alltypestiny c
   on b.id = c.id and b.int_col < 0 and a.int_col > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [INNER JOIN]
 |  hash predicates: c.id = b.id
 |  other predicates: b.int_col < 0
@@ -744,6 +792,8 @@ right outer join functional.alltypestiny b
 inner join functional.alltypestiny c
   on b.id = c.id and b.int_col < 0 and a.int_col > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [INNER JOIN]
 |  hash predicates: c.id = b.id
 |  other predicates: a.int_col > 10
@@ -773,6 +823,8 @@ full outer join functional.alltypestiny b
 inner join functional.alltypestiny c
   on b.id = c.id and b.int_col < 0 and a.int_col > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [INNER JOIN]
 |  hash predicates: b.id = c.id
 |  other predicates: a.int_col > 10, b.int_col < 0
@@ -805,6 +857,8 @@ inner join functional.alltypestiny d
 full outer join functional.alltypestiny e
   on d.id = e.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: e.id = d.id
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test b/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test
index c99a90b..deda7e9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/partition-key-scans.test
@@ -1,12 +1,16 @@
 # Test with aggregate expressions which ignore the distinct keyword.
 select min(month), max(year), ndv(day) from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: min(month), max(year), ndv(day)
 |
 00:UNION
    constant-operands=11
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: min(month), max(year), ndv(day)
 |
@@ -16,6 +20,8 @@ select min(month), max(year), ndv(day) from functional.alltypesagg
 # Test with explicit distinct keyword.
 select count(distinct year), ndv(day) from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(year), ndv:merge(day)
 |
@@ -26,6 +32,8 @@ select count(distinct year), ndv(day) from functional.alltypesagg
 00:UNION
    constant-operands=11
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(year), ndv:merge(day)
 |
@@ -39,6 +47,8 @@ select count(distinct year), ndv(day) from functional.alltypesagg
 # Test static partition pruning.
 select min(month), max(day) from functional.alltypesagg where year = 2010 and day = 1;
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: min(month), max(day)
 |
@@ -50,6 +60,8 @@ select c1, c2 from
   (select min(year) c1, max(month) c2, count(int_col) c3
    from functional.alltypes where year = 2000) t;
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: min(year), max(month)
 |
@@ -58,6 +70,8 @@ select c1, c2 from
 # Test with group by and having clauses.
 select ndv(month) from functional.alltypesagg group by year having max(day)=10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: ndv(month), max(day)
 |  group by: year
@@ -69,6 +83,8 @@ select ndv(month) from functional.alltypesagg group by year having max(day)=10
 # Test with group-by clauses (no aggregate expressions) only.
 select month from functional.alltypes group by month
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: month
 |
@@ -78,6 +94,8 @@ select month from functional.alltypes group by month
 # Test with distinct select list.
 select distinct month from functional.alltypes where month % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: month
 |
@@ -89,6 +107,8 @@ select min(a.month)
 from functional.alltypes as a, functional.alltypesagg as b
 where a.year = b.year
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: min(a.month)
 |
@@ -101,6 +121,8 @@ where a.year = b.year
 00:UNION
    constant-operands=24
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: min(a.month)
 |
@@ -122,6 +144,8 @@ select * from
   (select year, count(month) from functional.alltypes group by year) b
 on (a.year = b.year)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: year = year
 |  runtime filters: RF000 <- year
@@ -144,6 +168,8 @@ on (a.year = b.year)
 select min(a.year), ndv(b.timestamp_col) from
 functional.alltypes a, functional.alltypesnopart b
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: min(a.year), ndv(b.timestamp_col)
 |
@@ -161,6 +187,8 @@ select c1, c2 from
   (select ndv(a.year + b.year) c1, min(a.month + b.month) c2, count(a.int_col) c3 from
    functional.alltypes a, functional.alltypesagg b) t
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: ndv(a.year + b.year), min(a.month + b.month)
 |
@@ -179,6 +207,8 @@ from functional.alltypestiny t1 inner join
           min(t2.year) as int_col from functional.alltypestiny t2) t3
 on (t1.int_col = t3.int_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.int_col = min(t2.year)
 |  runtime filters: RF000 <- min(t2.year)
@@ -198,6 +228,8 @@ with c1 as (select distinct month from functional.alltypes),
      c2 as (select distinct year from functional.alltypes)
 select ndv(month) from (select * from c1 union all select * from c2) t
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: ndv(month)
 |
@@ -218,6 +250,8 @@ select ndv(month) from (select * from c1 union all select * from c2) t
 # If slots other than partition keys are accessed, make sure scan nodes are generated.
 select date_string_col, min(month) from functional.alltypes group by date_string_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: min(month)
 |  group by: date_string_col
@@ -228,6 +262,8 @@ select date_string_col, min(month) from functional.alltypes group by date_string
 # Make sure non-distinct aggregation functions will generate scan nodes.
 select count(month) from functional.alltypes
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(month)
 |
@@ -237,6 +273,8 @@ select count(month) from functional.alltypes
 # Make sure that queries without any aggregation will generate scan nodes.
 select month from functional.alltypes order by year
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SORT
 |  order by: year ASC
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test b/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
index 605f157..eeb9b97 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/predicate-propagation.test
@@ -3,6 +3,8 @@ select straight_join count(*)
 from functional.alltypes a join functional.alltypes b on (a.double_col = b.bigint_col)
 where b.bigint_col div 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -26,6 +28,8 @@ from
    inner join functional.alltypessmall b on (a.cnt = b.id)
 where b.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: count(id) = b.id
 |
@@ -45,6 +49,8 @@ where b.id < 10
 select count(*) from functional.alltypes
 where month = id and id = int_col and tinyint_col = int_col and int_col < 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -60,6 +66,8 @@ from
    left outer join (select id, string_col from functional.alltypes) b
    on (a.id = b.id and a.string_col = 'a' and b.string_col = 'b')
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: id = id
 |  other join predicates: string_col = 'a'
@@ -80,6 +88,8 @@ from
    on (a.id = b.id)
 where a.string_col = 'a' and b.string_col = 'b'
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: id = id
 |  other predicates: string_col = 'b'
@@ -99,6 +109,8 @@ from
    cross join (select id, string_col from functional.alltypes) b
 where a.string_col = 'a' and b.string_col = 'b'
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--01:SCAN HDFS [functional.alltypes]
@@ -109,6 +121,8 @@ where a.string_col = 'a' and b.string_col = 'b'
    partitions=24/24 files=24 size=478.45KB
    predicates: functional.alltypes.string_col = 'a'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
@@ -135,6 +149,8 @@ where c1 > 0
 order by 2, 1 desc
 limit 3
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=3]
 |  order by: c2 ASC, c1 DESC
 |
@@ -163,6 +179,8 @@ where c1 > 0
 order by 2, 1 desc
 limit 3
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=3]
 |  order by: c2 ASC, c1 DESC
 |
@@ -189,6 +207,8 @@ from functional.alltypes a
 where a.year = 2009 and b.month + 2 <= 4 and b.id = 17
   and cast(sin(c.int_col) as boolean) = true
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: b.id = c.id, b.month = c.month, b.year = c.year, b.smallint_col = c.int_col
 |  runtime filters: RF000 <- c.id, RF001 <- c.month, RF002 <- c.year, RF003 <- c.int_col
@@ -221,6 +241,8 @@ NODE 2:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -262,6 +284,8 @@ from (select * from functional.alltypes) a
 where a.year = 2009 and b.month + 2 <= 4 and b.id = 17
   and cast(sin(c.int_col) as boolean) = true
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypessmall.id = functional.alltypestiny.id, functional.alltypessmall.month = functional.alltypestiny.month, functional.alltypessmall.year = functional.alltypestiny.year, functional.alltypessmall.smallint_col = functional.alltypestiny.int_col
 |  runtime filters: RF000 <- functional.alltypestiny.id, RF001 <- functional.alltypestiny.month, RF002 <- functional.alltypestiny.year, RF003 <- functional.alltypestiny.int_col
@@ -294,6 +318,8 @@ NODE 2:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -340,6 +366,8 @@ from functional.alltypes a
    and a.month = b.month and b.month + 1 = 2)
 where a.year = 2009 and a.tinyint_col = 7 and a.id is null and b.id = 17 and b.int_col is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.id = b.id, a.month = b.month, a.tinyint_col = b.tinyint_col, a.year = b.year
 |  other predicates: b.int_col IS NULL, b.id = 17
@@ -368,6 +396,8 @@ NODE 0:
 NODE 1:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=1/090101.txt 0:1610
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
@@ -400,6 +430,8 @@ from functional.alltypessmall a
    and a.month = b.month and a.month + 1 = 2)
 where b.year = 2009 and b.tinyint_col = 7 and b.id is null and a.id = 17 and a.int_col is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: a.id = b.id, a.month = b.month, a.tinyint_col = b.tinyint_col, a.year = b.year
 |  other predicates: a.int_col IS NULL, a.id = 17
@@ -414,6 +446,8 @@ where b.year = 2009 and b.tinyint_col = 7 and b.id is null and a.id = 17 and a.i
    predicates: a.id = 17, a.tinyint_col = 7
    runtime filters: RF000 -> a.id, RF001 -> a.month, RF002 -> a.tinyint_col, RF003 -> a.year
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -448,6 +482,8 @@ from functional.alltypes a
   on (a.id = b.id and a.tinyint_col = b.int_col and a.year = b.year and a.month = b.month)
 where a.year = 2009 and b.month <= 2 and b.count_col + 1 = 17 and a.tinyint_col != 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = id, a.month = month, a.year = year, a.tinyint_col = int_col
 |  runtime filters: RF000 <- id, RF001 <- month, RF002 <- year, RF003 <- int_col
@@ -473,6 +509,8 @@ NODE 1:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=1/090101.txt 0:1610
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=2/090201.txt 0:1621
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
@@ -518,6 +556,8 @@ where a.id = b.id and
       b.count_col + 1 = 17 and
       a.tinyint_col != 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = id, a.month = month, a.year = year, a.tinyint_col = int_col
 |  runtime filters: RF000 <- id, RF001 <- month, RF002 <- year, RF003 <- int_col
@@ -536,6 +576,8 @@ where a.id = b.id and
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.id, RF001 -> a.month, RF002 -> a.year, RF003 -> a.tinyint_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
@@ -579,6 +621,8 @@ from functional.alltypes a
   on (a.id = b.id and a.tinyint_col = b.int_col and a.year = b.year and a.month = b.month)
 where a.year = 2009 and b.month <= 2 and b.count_col + 1 = 17 and a.tinyint_col != 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = id, a.month = month, a.year = year, a.tinyint_col = int_col
 |  runtime filters: RF000 <- id, RF001 <- month, RF002 <- year, RF003 <- int_col
@@ -609,6 +653,8 @@ NODE 1:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=3/090301.txt 0:1620
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=4/090401.txt 0:1621
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -661,6 +707,8 @@ where a.year = 2009 and
       a.year = b.year and
       a.month = b.month
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = id, a.month = month, a.year = year, a.tinyint_col = int_col
 |  runtime filters: RF000 <- id, RF001 <- month, RF002 <- year, RF003 <- int_col
@@ -682,6 +730,8 @@ where a.year = 2009 and
    predicates: a.id > 11, a.tinyint_col != 5
    runtime filters: RF000 -> a.id, RF001 -> a.month, RF002 -> a.year, RF003 -> a.tinyint_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -724,6 +774,8 @@ on (x.id = z.id)
 where x.year = 2009
 and z.month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypes.id = functional.alltypesagg.id
 |  runtime filters: RF000 <- functional.alltypesagg.id
@@ -756,6 +808,8 @@ where x.year = 2009
 and z.month = 1
 and x.id + x.b_id = 17
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = functional.alltypesagg.id
 |  runtime filters: RF000 <- functional.alltypesagg.id
@@ -785,6 +839,8 @@ from functional.alltypes a left outer join
 (select id, int_col from functional.alltypes group by 1, 2) b on (a.id = b.id)
 where a.id is null and isnull(b.id, 0) = 0 and b.int_col = 17
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.id = id
 |  other predicates: int_col = 17, isnull(id, 0) = 0
@@ -807,6 +863,8 @@ from functional.alltypes a left outer join
 on (a.id = b.id)
 where isnull(a.id, 0) = 0 and b.id is null  and b.int_col = 17
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.id = id
 |  other predicates: id IS NULL, int_col = 17
@@ -837,6 +895,8 @@ from
 right outer join functional.alltypes a on (a.id = b.id)
 where a.id is null and isnull(b.id, 0) = 0 and b.int_col = 17
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: id = a.id
 |  other predicates: int_col = 17, isnull(id, 0) = 0
@@ -861,6 +921,8 @@ from
 right outer join functional.alltypes a on (a.id = b.id)
 where isnull(a.id, 0) = 0 and b.id is null  and b.int_col = 17
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: id = a.id
 |  other predicates: id IS NULL, int_col = 17
@@ -893,6 +955,8 @@ select straight_join a.string_col from functional.alltypes a
 full outer join (select * from functional.alltypessmall where id > 0) b
 ON a.id=b.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: a.id = functional.alltypessmall.id
 |
@@ -916,6 +980,8 @@ left outer join
    where x.id is null) b
 on a.id=b.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.id = x.id
 |
@@ -942,6 +1008,8 @@ from functional.alltypes
 group by bool_col, int_col
 having bool_col = false and int_col > 0 and count(bigint_col) > 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(bigint_col)
 |  group by: bool_col, int_col
@@ -968,6 +1036,8 @@ and t2.id + t2.smallint_col + t2.bigint_col > 30
 # assigned in join, TODO: propagate multi-tuple predicates
 and t2.id + t3.int_col > 40
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t2.bigint_col = functional.alltypestiny.bigint_col, t2.id = functional.alltypestiny.id, t2.smallint_col = functional.alltypestiny.int_col
 |  other predicates: t2.id + functional.alltypestiny.int_col > 40
@@ -1009,6 +1079,8 @@ t1.id + t1.tinyint_col > 20
 # assigned in agg nodes in t2 and t3
 and t2.y + t2.z > 30
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: max(smallint_col) = max(smallint_col), min(int_col) = min(int_col)
 |  runtime filters: RF000 <- max(smallint_col)
@@ -1056,6 +1128,8 @@ and ifnull(t3.tinyint_col + t3.bigint_col, true) = true
 # assigned in scan of t1, t2 and t3
 and t1.id * t1.int_col < 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: t2.id = functional.alltypestiny.id, t2.int_col = functional.alltypestiny.int_col
 |  other predicates: functional.alltypestiny.tinyint_col + functional.alltypestiny.smallint_col + functional.alltypestiny.int_col > 10, ifnull(functional.alltypestiny.tinyint_col + functional.alltypestiny.bigint_col, TRUE) = TRUE
@@ -1084,6 +1158,8 @@ functional.alltypes t1 inner join functional.alltypessmall t2
 on (t1.id = t2.month and t1.year = t2.year and t1.month = t2.month)
 where t2.year + t2.month > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t2.month, t1.year = t2.year
 |  runtime filters: RF000 <- t2.month, RF001 <- t2.year
@@ -1107,6 +1183,8 @@ on (t1.id = t2.id
     and t1.tinyint_col = t2.int_col)
 where t1.id + t1.tinyint_col > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t2.id, t1.tinyint_col = t2.tinyint_col
 |  runtime filters: RF000 <- t2.id, RF001 <- t2.tinyint_col
@@ -1132,6 +1210,8 @@ inner join (select bigint_col, min(int_col) x, max(int_col) y
 on (t1.id = t2.bigint_col and t1.int_col = t2.x)
 where t1.id + t1.int_col > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.int_col = min(int_col), t1.id = bigint_col
 |  runtime filters: RF000 <- min(int_col), RF001 <- bigint_col
@@ -1157,6 +1237,8 @@ left anti join
 on (a.id = b.id)
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT ANTI JOIN]
 |  hash predicates: a.id = id
 |
@@ -1176,6 +1258,8 @@ right anti join functional.alltypestiny b
 on (a.id = b.id)
 where b.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT ANTI JOIN]
 |  hash predicates: id = b.id
 |
@@ -1197,6 +1281,8 @@ select * from
    group by j.int_col) v
 where v.int_col = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(a.tinyint_col)
 |  group by: b.int_col
@@ -1223,6 +1309,8 @@ select * from
     on a.id = b.id) j) v
 where v.int_col = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: b.id = a.id
 |  other predicates: b.int_col = 10
@@ -1245,6 +1333,8 @@ SELECT count(*) FROM
   WHERE n_name = 'BRAZIL' AND n_regionkey = 1 AND c_custkey % 2 = 0) cn
  LEFT OUTER JOIN tpch_parquet.region r ON n_regionkey = r_regionkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-propagation.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-propagation.test b/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-propagation.test
index f1abd1d..446aead 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-propagation.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/runtime-filter-propagation.test
@@ -2,6 +2,8 @@
 select straight_join * from functional.alltypesagg t1, functional.alltypesnopart t2
 where t1.year = t2.int_col and t2.id < 10 and t1.id = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year = t2.int_col
 |  runtime filters: RF000 <- t2.int_col
@@ -15,6 +17,8 @@ where t1.year = t2.int_col and t2.id < 10 and t1.id = 10
    predicates: t1.id = 10
    runtime filters: RF000 -> t1.year
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -38,6 +42,8 @@ select straight_join * from functional.alltypestiny t1, functional.alltypesagg t
 where t1.year = t2.int_col and t3.tinyint_col = t2.id and t3.month = t4.id and
   t2.bool_col = true and t4.bigint_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: t3.month = t4.id
 |  runtime filters: RF000 <- t4.id
@@ -71,6 +77,8 @@ where t1.year = t2.int_col and t3.tinyint_col = t2.id and t3.month = t4.id and
 select straight_join * from functional.alltypesagg t1, functional.alltypesnopart t2
 where t1.year = t2.int_col and t1.month = t2.bigint_col and t2.id = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year = t2.int_col, t1.month = t2.bigint_col
 |  runtime filters: RF000 <- t2.int_col, RF001 <- t2.bigint_col
@@ -88,6 +96,8 @@ select straight_join * from functional.alltypesagg t1,
   (select * from functional.alltypesnopart t2 where t2.id = 1) v
 where t1.year = v.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year = t2.int_col
 |  runtime filters: RF000 <- t2.int_col
@@ -107,6 +117,8 @@ select straight_join * from functional.alltypesagg t1,
    where t2.bigint_col < 10) v
 where v.id1 = t1.year
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year = id + int_col
 |  runtime filters: RF000 <- id + int_col
@@ -123,6 +135,8 @@ where v.id1 = t1.year
 select straight_join * from functional.alltypesagg t1, functional.alltypesnopart t2
 where t1.year + 1 = t2.id and t2.int_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year + 1 = t2.id
 |  runtime filters: RF000 <- t2.id
@@ -141,6 +155,8 @@ where t1.id = t2.id and t1.year + t2.int_col = t1.month + t2.tinyint_col
 and t1.year = t1.month + t2.int_col and t1.year + t2.smallint_col = t2.tinyint_col
 and t1.int_col = 1 and 1 = t2.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t2.id
 |  other predicates: t1.year + t2.smallint_col = t2.tinyint_col, t1.year = t1.month + t2.int_col, t1.year + t2.int_col = t1.month + t2.tinyint_col
@@ -161,6 +177,8 @@ select straight_join * from functional.alltypesagg t1, functional.alltypesnopart
 where t1.year + t1.month = t2.id and t1.int_col + 1 - t1.tinyint_col = t2.smallint_col + 10
 and t1.int_col * 100 = t2.bigint_col / 100 and t2.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year + t1.month = t2.id, t1.int_col * 100 = t2.bigint_col / 100, t1.int_col + 1 - t1.tinyint_col = t2.smallint_col + 10
 |  runtime filters: RF000 <- t2.id, RF001 <- t2.bigint_col / 100, RF002 <- t2.smallint_col + 10
@@ -182,6 +200,8 @@ select straight_join * from
   functional.alltypesnopart t3
 where v.year = t3.int_col and t3.bool_col = true
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year + t2.year = t3.int_col
 |
@@ -207,6 +227,8 @@ select straight_join * from functional.alltypesagg t1,
    having count(int_col) < 10) v
 where v.cnt = t1.year and v.id = t1.month
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.month = id, t1.year = count(int_col)
 |  runtime filters: RF000 <- id, RF001 <- count(int_col)
@@ -230,6 +252,8 @@ select straight_join * from functional.alltypesagg t1,
    functional.alltypesnopart t3 where t2.int_col = t3.int_col) v
 where v.id = t1.year and t1.month = v.tinyint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.month = t3.tinyint_col, t1.year = t2.id + t3.id
 |  runtime filters: RF000 <- t3.tinyint_col, RF001 <- t2.id + t3.id
@@ -257,6 +281,8 @@ select straight_join * from functional.alltypesagg t1,
    where t2.id = t3.id and t3.int_col = t4.int_col and t4.tinyint_col = t2.tinyint_col) v
 where t1.year = v.int_col and t1.year = v.id and t1.month = v.tinyint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year = t2.int_col, t1.month = t4.tinyint_col
 |  runtime filters: RF000 <- t2.int_col, RF001 <- t4.tinyint_col
@@ -291,6 +317,8 @@ select straight_join * from functional.alltypesagg t1, functional.alltypesnopart
 where t1.year = t2.id and t1.year = t3.int_col and t1.year = t4.tinyint_col and
   t2.bool_col = false and t3.bool_col = true and t4.bigint_col in (1,2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.year = t4.tinyint_col
 |  runtime filters: RF000 <- t4.tinyint_col
@@ -327,6 +355,8 @@ select straight_join * from functional.alltypesagg t1, functional.alltypesnopart
 where t1.year = t2.id and t2.int_col = t3.tinyint_col and t3.month = t4.bigint_col
   and t4.smallint_col = t5.smallint_col and t5.id = t1.month
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.month = t5.id, t4.smallint_col = t5.smallint_col
 |  runtime filters: RF000 <- t5.id, RF001 <- t5.smallint_col
@@ -368,6 +398,8 @@ select straight_join * from functional.alltypesagg t1 left outer join functional
   on t1.year = t2.int_col
 where t2.id = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: t1.year = t2.int_col
 |  other predicates: t2.id = 1
@@ -385,6 +417,8 @@ select straight_join * from functional.alltypesagg t1 left outer join functional
   on t1.year = t2.int_col
 where t2.id = 2 and t1.month = t2.tinyint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: t1.year = t2.int_col
 |  other predicates: t2.id = 2, t1.month = t2.tinyint_col
@@ -406,6 +440,8 @@ select straight_join * from functional.alltypesagg t1 left outer join functional
   on t1.year = t5.smallint_col
 where t2.id = 1 and t3.int_col = 1 and t4.bool_col = true and t5.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: t1.year = t5.smallint_col
 |  other predicates: t2.id = 1, t3.int_col = 1, t4.bool_col = TRUE
@@ -449,6 +485,8 @@ from functional.alltypesagg t1 right outer join functional.alltypesnopart t2
   on t1.year = t2.int_col and t1.month = 1 and t2.int_col = 10
 where t2.id = 10 and t1.month = t2.tinyint_col and t1.int_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: t1.year = t2.int_col
 |  other join predicates: t2.int_col = 10
@@ -470,6 +508,8 @@ select straight_join * from functional.alltypesagg t1 left semi join functional.
   on t1.month = t3.tinyint_col
 where t3.id = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: t1.month = t3.tinyint_col
 |  runtime filters: RF000 <- t3.tinyint_col
@@ -494,6 +534,8 @@ select straight_join * from functional.alltypesagg t1
 where t1.year not in (select id from functional.alltypesnopart where int_col = 10)
 and t1.int_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
 |  hash predicates: t1.year = id
 |
@@ -512,6 +554,8 @@ select straight_join * from
   (select id, int_col from functional.alltypesnopart where tinyint_col < 10) v2
 where v1.year = v2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: year = id
 |  runtime filters: RF000 <- id
@@ -535,6 +579,8 @@ select straight_join * from
   functional.alltypes t2
 where v1.cnt = t2.id and t2.int_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: count(*) = t2.id
 |
@@ -559,6 +605,8 @@ select straight_join * from
    functional.alltypesnopart t3
 where v2.year = t3.smallint_col and t3.id = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: v1.year = t3.smallint_col
 |  runtime filters: RF000 <- t3.smallint_col
@@ -593,6 +641,8 @@ join functional.alltypestiny b on v.year = b.year
 join functional.alltypestiny c on v.year = c.year
 where b.int_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:HASH JOIN [INNER JOIN]
 |  hash predicates: year = c.year
 |  runtime filters: RF000 <- c.year
@@ -631,6 +681,8 @@ select straight_join * from
   functional.alltypesnopart t2
 where v1.year = t2.id and v1.int_col = t2.int_col and t2.smallint_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: int_col = t2.int_col, year = t2.id
 |
@@ -651,6 +703,8 @@ select straight_join * from
   functional.alltypesnopart t3
 where v.year = t3.int_col and t3.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: year = t3.int_col
 |  runtime filters: RF000 <- t3.int_col
@@ -680,6 +734,8 @@ select straight_join count(*) from
   on a.month = b.month
 where b.int_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -716,6 +772,8 @@ select straight_join count(*) from
   on a.month = b.month
 where b.int_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -751,6 +809,8 @@ union all
 select straight_join t4.id, t3.year from functional.alltypes t3, functional.alltypesnopart t4
 where t3.month = t4.smallint_col and t4.bool_col = true
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--06:HASH JOIN [INNER JOIN]
@@ -786,6 +846,8 @@ select straight_join count(*) from functional.alltypes a
   on a.id = b.id
 where (b.id - b.id) < 1 AND (b.int_col - b.int_col) < 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -817,6 +879,8 @@ select straight_join * from
    functional.alltypesnopart t3
 where v1.month = t3.tinyint_col and v1.year = t3.id and t3.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: year = t3.id, month = t3.tinyint_col
 |
@@ -852,6 +916,8 @@ select straight_join * from
    from functional.alltypes) v, functional.alltypestiny v1
 where v.year = v1.int_col and v.year = 2009
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: year = v1.int_col
 |
@@ -887,6 +953,8 @@ select straight_join * from
   ) v3
 where v2.month = v3.intcol1
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.month = t4.int_col
 |  runtime filters: RF000 <- t4.int_col
@@ -927,6 +995,8 @@ select straight_join 1 from functional.alltypestiny t1 join functional.alltypest
   join functional.alltypestiny t3 on t2.id = t3.id
 where t3.int_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t2.id = t3.id
 |  runtime filters: RF000 <- t3.id
@@ -952,6 +1022,8 @@ select straight_join 1 from functional.alltypestiny t1 join functional.alltypest
   join functional.alltypestiny t3 on t1.id = t3.id
 where t3.int_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t3.id
 |  runtime filters: RF000 <- t3.id
@@ -979,6 +1051,8 @@ select straight_join 1 from tpch_nested_parquet.customer c,
    on o1.o_orderkey = o2.o_orderkey) v
 where c_custkey = v.o_orderkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--06:HASH JOIN [INNER JOIN]
@@ -1002,6 +1076,8 @@ from functional.alltypestiny t1 join
   (select * from functional.alltypessmall t2 where false) v on t1.id = v.id
 where v.int_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t2.id
 |  runtime filters: RF000 <- t2.id
@@ -1018,6 +1094,8 @@ select straight_join 1 from
   (select * from functional.alltypestiny where false) v1 join
   (select * from functional.alltypessmall where false) v2 on v1.id = v2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypestiny.id = functional.alltypessmall.id
 |
@@ -1032,6 +1110,8 @@ select straight_join 1 from functional.alltypestiny t1 join functional.alltypest
   join functional.alltypestiny t3 on t1.id = t3.id
   join functional.alltypestiny t4 on t1.id + t2.id = t4.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id + t2.id = t4.id
 |  runtime filters: RF000 <- t4.id
@@ -1066,6 +1146,8 @@ select straight_join 1 from functional.alltypestiny a1
   inner join functional.alltypestiny a3 ON a3.smallint_col = a1.int_col
   inner join functional.alltypes a4 ON a4.smallint_col = a3.smallint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: a3.smallint_col = a4.smallint_col
 |  runtime filters: RF000 <- a4.smallint_col
@@ -1096,6 +1178,8 @@ from functional.alltypestiny t1 left join
   on t2.int_col = t1.month
 where t1.month is not null
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  group by: t1.int_col
 |
@@ -1210,6 +1294,8 @@ from big_six
   inner join big_three
   inner join small_four_2
 ---- PLAN
+PLAN-ROOT SINK
+|
 36:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--28:HASH JOIN [INNER JOIN]

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/small-query-opt.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/small-query-opt.test b/testdata/workloads/functional-planner/queries/PlannerTest/small-query-opt.test
index 82b8d73..60e9dd9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/small-query-opt.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/small-query-opt.test
@@ -1,9 +1,13 @@
 select * from functional_seq.alltypes t1 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional_seq.alltypes t1]
    partitions=24/24 files=24 size=562.59KB
    limit: 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional_seq.alltypes t1]
    partitions=24/24 files=24 size=562.59KB
    limit: 5
@@ -11,10 +15,14 @@ select * from functional_seq.alltypes t1 limit 5
 # Query is over the limit of 8 rows to be optimized, will distribute the query
 select * from functional.alltypes t1 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes t1]
    partitions=24/24 files=24 size=478.45KB
    limit: 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |  limit: 10
 |
@@ -25,10 +33,14 @@ select * from functional.alltypes t1 limit 10
 # Query is optimized, run on coordinator only
 select * from functional.alltypes t1 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes t1]
    partitions=24/24 files=24 size=478.45KB
    limit: 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes t1]
    partitions=24/24 files=24 size=478.45KB
    limit: 5
@@ -36,11 +48,15 @@ select * from functional.alltypes t1 limit 5
 # If a predicate is applied the optimization is disabled
 select * from functional.alltypes t1 where t1.id < 99 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes t1]
    partitions=24/24 files=24 size=478.45KB
    predicates: t1.id < 99
    limit: 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |  limit: 5
 |
@@ -52,10 +68,14 @@ select * from functional.alltypes t1 where t1.id < 99 limit 5
 # No optimization for hbase tables
 select * from functional_hbase.alltypes t1 where t1.id < 99 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypes t1]
    predicates: t1.id < 99
    limit: 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |  limit: 5
 |
@@ -66,9 +86,13 @@ select * from functional_hbase.alltypes t1 where t1.id < 99 limit 5
 # Applies optimization for small queries in hbase
 select * from functional_hbase.alltypes t1 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypes t1]
    limit: 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypes t1]
    limit: 5
 ====
@@ -122,6 +146,8 @@ select * from functional_hbase.alltypes limit 5
 union all
 select * from functional_hbase.alltypes limit 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HBASE [functional_hbase.alltypes]
@@ -130,6 +156,8 @@ select * from functional_hbase.alltypes limit 2
 01:SCAN HBASE [functional_hbase.alltypes]
    limit: 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HBASE [functional_hbase.alltypes]
@@ -142,6 +170,8 @@ select * from functional_hbase.alltypes limit 5
 union all
 select * from functional_hbase.alltypes limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HBASE [functional_hbase.alltypes]
@@ -150,6 +180,8 @@ select * from functional_hbase.alltypes limit 5
 01:SCAN HBASE [functional_hbase.alltypes]
    limit: 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--04:EXCHANGE [UNPARTITIONED]
@@ -168,6 +200,8 @@ select * from functional_hbase.alltypes limit 5
 select * from
   functional.testtbl a join functional.testtbl b on a.id = b.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -186,6 +220,8 @@ select * from
 select * from
   functional.testtbl a, functional.testtbl b
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
@@ -202,6 +238,8 @@ select * from
   functional.alltypestiny a
 where a.id in (select id from functional.alltypestiny limit 5) limit 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |  limit: 5
 |
@@ -237,6 +275,8 @@ select id, bool_col
 from functional.alltypestiny c
 where year=2009 and month=2
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:UNION
 |
 |--03:AGGREGATE [FINALIZE]
@@ -256,6 +296,8 @@ where year=2009 and month=2
 # IMPALA-2527: Tests that the small query optimization is disabled for colleciton types
 select key from functional.allcomplextypes.map_map_col.value limit 5;
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |  limit: 5
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test b/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
index 21776ba..060d470 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/subquery-rewrite.test
@@ -4,6 +4,8 @@ from functional.alltypes
 where id in
   (select id from functional.alltypesagg)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = id
 |  runtime filters: RF000 <- id
@@ -21,6 +23,8 @@ from functional.alltypes
 where id not in
   (select id from functional.alltypesagg)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
 |  hash predicates: id = id
 |
@@ -39,6 +43,8 @@ where a.int_col not in
    where g.id = a.id and g.bigint_col < a.bigint_col)
 and a.int_col < 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
 |  hash predicates: a.int_col = int_col
 |  other join predicates: a.id = g.id, g.bigint_col < a.bigint_col
@@ -56,6 +62,8 @@ select *
 from functional.alltypes a
 where a.id not in (select id from functional.alltypes b where a.id = b.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
 |  hash predicates: a.id = id
 |  other join predicates: a.id = b.id
@@ -73,6 +81,8 @@ where int_col in
   (select int_col from functional.alltypesagg g where a.id = g.id and g.bigint_col < 10)
 and bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -95,6 +105,8 @@ from functional.alltypes t
 where t.int_col + 1 in
   (select int_col + bigint_col from functional.alltypesagg)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t.int_col + 1 = int_col + bigint_col
 |  runtime filters: RF000 <- int_col + bigint_col
@@ -114,6 +126,8 @@ where t.id in
 and t.tinyint_col not in (select tinyint_col from functional.alltypestiny)
 and t.bigint_col < 1000
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
 |  hash predicates: t.tinyint_col = tinyint_col
 |
@@ -139,6 +153,8 @@ from functional.alltypesagg a, functional.alltypes t
 where a.id = t.id and a.int_col in
   (select int_col from functional.alltypestiny where bool_col = false)
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -170,6 +186,8 @@ where a.id in
    where s.int_col = t.int_col and a.bool_col = s.bool_col)
 and a.int_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -201,6 +219,8 @@ where a.id in
   (select id from functional.alltypestiny)
 and t.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -231,6 +251,8 @@ from functional.alltypes a left outer join
 on a.int_col = t.int_col
 where a.bool_col = false and t.bigint_col < 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -262,6 +284,8 @@ where a.int_col in
    from functional.alltypes t, functional.alltypessmall s, functional.alltypestiny n
    where t.id = s.id and s.bigint_col = n.bigint_col and n.bool_col = false)
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:AGGREGATE [FINALIZE]
 |  output: count(id)
 |
@@ -304,6 +328,8 @@ where t.id in
    (select id, count(*) as cnt from functional.alltypessmall group by id) s
    where s.id = a.id and s.cnt = 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t.id = a.id
 |  runtime filters: RF000 <- a.id
@@ -332,6 +358,8 @@ with t as (select a.* from functional.alltypes a where id in
   (select id from functional.alltypestiny))
 select * from t where t.bool_col = false and t.int_col = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = id
 |  runtime filters: RF000 <- id
@@ -354,6 +382,8 @@ where s.string_col = t.string_col and t.int_col in
   (select int_col from functional.alltypessmall)
 and s.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: a.int_col = int_col
 |  runtime filters: RF000 <- int_col
@@ -395,6 +425,8 @@ where id in
   (select id from functional.alltypesagg a where t.int_col = a.int_col)
 and t.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = id, t.int_col = a.int_col
 |  runtime filters: RF000 <- id, RF001 <- a.int_col
@@ -416,6 +448,8 @@ where id in
    and bool_col = false)
 and bigint_col < 1000
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = id
 |  runtime filters: RF000 <- id
@@ -446,6 +480,8 @@ where id in
     (select tinyint_col from functional.alltypestiny s
      where s.bigint_col = a.bigint_col))
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = id, t.int_col = a.int_col
 |  runtime filters: RF000 <- id, RF001 <- a.int_col
@@ -472,6 +508,8 @@ where id in
   (select id from functional.alltypesagg a where a.int_col in
     (select int_col from functional.alltypestiny s where a.bigint_col = s.bigint_col))
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = id
 |  runtime filters: RF000 <- id
@@ -497,6 +535,8 @@ from functional.alltypes
 where id in
   (select id from functional.alltypes where id < 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = id
 |  runtime filters: RF000 <- id
@@ -516,6 +556,8 @@ from functional.alltypesagg a inner join functional.alltypes t on t.id = a.id
 where t.int_col < 10 and t.int_col in
   (select int_col from functional.alltypessmall s where s.id = t.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t.id = s.id, t.int_col = int_col
 |  runtime filters: RF000 <- s.id, RF001 <- int_col
@@ -543,6 +585,8 @@ from functional.alltypes t
 where exists
   (select * from functional.alltypesagg a where a.id = t.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -567,6 +611,8 @@ where exists
    group by id, int_col, bool_col)
 and tinyint_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: a.tinyint_col = b.tinyint_col
 |  runtime filters: RF000 <- b.tinyint_col
@@ -588,6 +634,8 @@ from functional.alltypes t
 where not exists
   (select id from functional.alltypesagg a where t.int_col = a.int_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -610,6 +658,8 @@ where not exists
    group by b.id, b.int_col, b.bigint_col)
 and bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -640,6 +690,8 @@ select *
 from functional.alltypestiny t
 where exists (select * from functional.alltypessmall s where s.id < 5)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [LEFT SEMI JOIN]
 |
 |--01:SCAN HDFS [functional.alltypessmall s]
@@ -658,6 +710,8 @@ where exists
    from functional.alltypesagg where tinyint_col = 10
    group by id, int_col, bigint_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:NESTED LOOP JOIN [RIGHT SEMI JOIN]
 |
 |--00:SCAN HDFS [functional.alltypestiny t]
@@ -676,6 +730,8 @@ select 1
 from functional.alltypestiny t
 where exists (select * from functional.alltypessmall limit 0)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Uncorrelated NOT EXISTS
@@ -683,6 +739,8 @@ select *
 from functional.alltypestiny t
 where not exists (select * from functional.alltypessmall s where s.id < 5)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [LEFT ANTI JOIN]
 |
 |--01:SCAN HDFS [functional.alltypessmall s]
@@ -701,6 +759,8 @@ select *
 from w1 t
 where not exists (select 1 from w2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [LEFT ANTI JOIN]
 |
 |--01:SCAN HDFS [functional.alltypessmall s]
@@ -719,6 +779,8 @@ where not exists
    from functional.alltypesagg where tinyint_col = 10
    group by id, int_col, bigint_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:NESTED LOOP JOIN [RIGHT ANTI JOIN]
 |
 |--00:SCAN HDFS [functional.alltypestiny t]
@@ -737,6 +799,8 @@ select 1
 from functional.alltypestiny t
 where not exists (select * from functional.alltypessmall limit 0)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypestiny t]
    partitions=4/4 files=4 size=460B
 ====
@@ -748,6 +812,8 @@ where exists
     (select * from functional.alltypesagg g where g.int_col = t.int_col
      and g.bool_col = false))
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -790,6 +856,8 @@ and g.tinyint_col <
 group by g.int_col
 having count(*) < 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: g.int_col
@@ -847,6 +915,8 @@ where a.int_col in
   group by int_col)
 and a.bigint_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: a.int_col = count(int_col)
 |  runtime filters: RF000 <- count(int_col)
@@ -871,6 +941,8 @@ where a.int_col <
   (select max(int_col) from functional.alltypesagg g where g.bool_col = true)
 and a.bigint_col > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: a.int_col < max(int_col)
 |
@@ -891,6 +963,8 @@ from functional.alltypesagg a
 where (select max(id) from functional.alltypes t where t.bool_col = false) > 10
 and a.int_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--02:AGGREGATE [FINALIZE]
@@ -915,6 +989,8 @@ where a.id =
 and a.bool_col = false
 group by a.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: a.int_col
@@ -947,6 +1023,8 @@ where t.int_col <
 and a.bool_col = false
 group by t.tinyint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: t.tinyint_col
@@ -993,6 +1071,8 @@ and a.tinyint_col >
   (select max(tinyint_col) from functional.alltypessmall s where s.id < 10)
 and t.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: a.tinyint_col > max(tinyint_col)
 |
@@ -1041,6 +1121,8 @@ where t.int_col <
       where a.id = g.id
       and a.bool_col = false))
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t.id = g.id
 |  other join predicates: t.int_col < avg(g.int_col) * 2
@@ -1079,6 +1161,8 @@ where a.int_col <
    where s.id = a.id and s.tinyint_col >
      (select count(*) from functional.alltypestiny where bool_col = false))
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.id = s.id
 |  other predicates: a.int_col < zeroifnull(count(*))
@@ -1115,6 +1199,8 @@ from functional.alltypesagg g
 where 100 < (select count(*) from functional.alltypes where bool_col = false and id < 5)
 and bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  group by: id, bool_col
 |
@@ -1138,6 +1224,8 @@ from functional.alltypesagg g
 where 100 > (select count(distinct id) from functional.alltypestiny where int_col < 5)
 and g.bigint_col < 1000 and g.bigint_col = true
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  group by: g.id, g.bool_col, g.tinyint_col, g.smallint_col, g.int_col, g.bigint_col, g.float_col, g.double_col, g.date_string_col, g.string_col, g.timestamp_col, g.year, g.month, g.day
 |
@@ -1163,6 +1251,8 @@ select *
 from functional.alltypestiny t
 where (select max(int_col) from functional.alltypesagg where int_col is null) is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--02:AGGREGATE [FINALIZE]
@@ -1183,6 +1273,8 @@ where (select count(*) from functional.alltypesagg g where t.id = g.id) is null
 and bool_col = false
 group by int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: int_col
@@ -1214,6 +1306,8 @@ where
    where g.id = t.id and g.int_col is null) is null
 and t.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: g.id = t.id
 |  runtime filters: RF000 <- t.id
@@ -1239,6 +1333,8 @@ where 1 +
   (select count(*) from functional.alltypesagg where bool_col = false) = t.int_col + 2
 and t.bigint_col < 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t.int_col + 2 = 1 + count(*)
 |  runtime filters: RF000 <- 1 + count(*)
@@ -1262,6 +1358,8 @@ where nullifzero((select min(id) from functional.alltypessmall s where s.bool_co
   is null
 and t.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--02:AGGREGATE [FINALIZE]
@@ -1286,6 +1384,8 @@ where t.int_col <
    limit 1)
 group by t.bool_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: min(t.id)
 |  group by: t.bool_col
@@ -1313,6 +1413,8 @@ where int_col between
   (select min(int_col) from functional.alltypessmall where bool_col = false) and
   (select max(int_col) from functional.alltypessmall where bool_col = true)
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: int_col <= max(int_col)
 |
@@ -1344,6 +1446,8 @@ where
    from functional.alltypestiny tt1
    where t1.id = tt1.month) < t1.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: tt1.month = t1.id
 |  other predicates: zeroifnull(count(tt1.smallint_col)) < t1.id
@@ -1369,6 +1473,8 @@ where
   < 10
 group by int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: int_col
@@ -1398,6 +1504,8 @@ where
 and bool_col = false
 group by int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: int_col
@@ -1425,6 +1533,8 @@ select 1
 from functional.alltypestiny t1
 where (select count(*) from functional.alltypessmall) + t1.int_col = t1.bigint_col - 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: count(*) + t1.int_col = t1.bigint_col - 1
 |
@@ -1443,6 +1553,8 @@ select 1
 from functional.alltypestiny t1 join functional.alltypessmall t2 on t1.id = t2.id
 where (select count(*) from functional.alltypes) + 1 = t1.int_col + t2.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.int_col + t2.int_col = count(*) + 1
 |
@@ -1470,6 +1582,8 @@ from functional.alltypestiny t1 join functional.alltypessmall t2 on t1.id = t2.i
 where
   (select count(*) from functional.alltypes) + t2.bigint_col = t1.int_col + t2.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: count(*) + t2.bigint_col = t1.int_col + t2.int_col
 |
@@ -1498,6 +1612,8 @@ where
    from functional.alltypesagg t1 inner join functional.alltypes t2 on t1.id = t2.id
    where t1.id + t2.id = t.int_col) = t.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: sum(t1.id) = t.int_col
 |
@@ -1528,6 +1644,8 @@ where
    from functional.alltypesagg t1 inner join functional.alltypes t2 on t1.id = t2.id
    where t1.id + t2.id = t.bigint_col) = t.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: t1.id + t2.id = t.bigint_col, sum(t1.id) = t.int_col
 |
@@ -1560,6 +1678,8 @@ where
    on tt1.int_col = tt2.int_col
    where tt1.id + tt2.id = t1.int_col - t2.int_col) = t1.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: sum(tt1.id) = t1.bigint_col, tt1.id + tt2.id = t1.int_col - t2.int_col
 |
@@ -1602,6 +1722,8 @@ and not exists
    from functional.alltypesagg t3
    where t1.id = t3.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Correlated EXISTS and NOT EXISTS subqueries with limit 0 and
@@ -1625,6 +1747,8 @@ and not exists
    from functional.alltypestiny t5
    where t1.id = t5.id and false)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypestiny t1]
    partitions=4/4 files=4 size=460B
 ====
@@ -1647,6 +1771,8 @@ and not exists
    where t4.int_col = t1.tinyint_col
    having count(id) > 200)
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [LEFT ANTI JOIN]
 |  hash predicates: t1.tinyint_col = t4.int_col
 |
@@ -1680,6 +1806,8 @@ where t1.id is not distinct from
 (select min(id) from functional.alltypes t2
 where t1.int_col is not distinct from t2.int_col);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.id IS NOT DISTINCT FROM min(id), t1.int_col IS NOT DISTINCT FROM t2.int_col
 |  runtime filters: RF000 <- min(id), RF001 <- t2.int_col
@@ -1700,6 +1828,8 @@ where t1.id is distinct from
 (select min(id) from functional.alltypes t2
 where t1.int_col is not distinct from t2.int_col);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.int_col IS NOT DISTINCT FROM t2.int_col
 |  other join predicates: t1.id IS DISTINCT FROM min(id)
@@ -1721,6 +1851,8 @@ where t1.id =
 (select min(id) from functional.alltypes t2
 where t1.int_col is not distinct from t2.int_col);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.id = min(id), t1.int_col IS NOT DISTINCT FROM t2.int_col
 |  runtime filters: RF000 <- min(id), RF001 <- t2.int_col
@@ -1741,6 +1873,8 @@ where t1.id !=
 (select min(id) from functional.alltypes t2
 where t1.int_col is not distinct from t2.int_col);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.int_col IS NOT DISTINCT FROM t2.int_col
 |  other join predicates: t1.id != min(id)
@@ -1762,6 +1896,8 @@ where t1.id is not distinct from
 (select min(id) from functional.alltypes t2
 where t1.int_col = t2.int_col);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.id IS NOT DISTINCT FROM min(id), t1.int_col = t2.int_col
 |  runtime filters: RF000 <- min(id), RF001 <- t2.int_col
@@ -1782,6 +1918,8 @@ where t1.id is distinct from
 (select min(id) from functional.alltypes t2
 where t1.int_col = t2.int_col);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.int_col = t2.int_col
 |  other join predicates: t1.id IS DISTINCT FROM min(id)
@@ -1803,6 +1941,8 @@ where t1.id =
 (select min(id) from functional.alltypes t2
 where t1.int_col = t2.int_col);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.id = min(id), t1.int_col = t2.int_col
 |  runtime filters: RF000 <- min(id), RF001 <- t2.int_col
@@ -1823,6 +1963,8 @@ where t1.id !=
 (select min(id) from functional.alltypes t2
 where t1.int_col = t2.int_col);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: t1.int_col = t2.int_col
 |  other join predicates: t1.id != min(id)
@@ -1850,6 +1992,8 @@ select 1 from functional.alltypes t where id in
    a.double_col between round(acos(t.float_col), 2)
                 and cast(t.string_col as int))
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: id = id
 |  other join predicates: a.tinyint_col >= t.tinyint_col, t.float_col >= a.float_col, a.smallint_col <= t.int_col, a.tinyint_col <= t.smallint_col, t.float_col <= a.double_col, a.double_col <= CAST(t.string_col AS INT), t.string_col >= a.string_col, a.double_col >= round(acos(t.float_col), 2)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/topn.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/topn.test b/testdata/workloads/functional-planner/queries/PlannerTest/topn.test
index e9dcd43..4252ac7 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/topn.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/topn.test
@@ -3,12 +3,16 @@ from functional.testtbl
 order by name
 limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=1]
 |  order by: name ASC
 |
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: name ASC
 |  limit: 1
@@ -26,6 +30,8 @@ group by 1
 order by 2 desc
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=10]
 |  order by: count(*) DESC
 |
@@ -37,6 +43,8 @@ limit 10
    partitions=1/1 files=0 size=0B
    predicates: name LIKE 'm%'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC
 |  limit: 10
@@ -65,6 +73,8 @@ group by 1
 order by 2
 limit 4
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=4]
 |  order by: sum(float_col) ASC
 |
@@ -75,6 +85,8 @@ limit 4
 00:SCAN HBASE [functional_hbase.alltypessmall]
    predicates: id < 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(float_col) ASC
 |  limit: 4
@@ -101,8 +113,12 @@ group by 1
 order by 2,3 desc
 limit 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Test correct identification of the implicit aliasing of int_col in the select
@@ -114,6 +130,8 @@ where t1.id = t2.id and t2.int_col is not null
 order by int_col
 limit 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=2]
 |  order by: int_col ASC
 |
@@ -129,6 +147,8 @@ limit 2
    partitions=4/4 files=4 size=6.32KB
    runtime filters: RF000 -> t1.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: int_col ASC
 |  limit: 2
@@ -153,12 +173,16 @@ limit 2
 # Test that the top-n is on int_col and not on the id column
 select int_col as id from functional.alltypessmall order by id limit 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=2]
 |  order by: int_col ASC
 |
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: int_col ASC
 |  limit: 2
@@ -172,12 +196,16 @@ select int_col as id from functional.alltypessmall order by id limit 2
 # Test that the top-n is on id and not on int_col
 select int_col as id from functional.alltypessmall order by functional.alltypessmall.id limit 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=2]
 |  order by: id ASC
 |
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: id ASC
 |  limit: 2
@@ -196,6 +224,8 @@ where t1.id = t2.id and t2.int_col is not null
 order by int_col
 limit 10 offset 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=10 OFFSET=5]
 |  order by: int_col ASC
 |
@@ -222,6 +252,8 @@ NODE 1:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=3/090301.txt 0:1620
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=4/090401.txt 0:1621
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  offset: 5
 |  order by: int_col ASC
@@ -251,6 +283,8 @@ select int_col, bigint_col from
    select * from functional.alltypessmall) t
 order by int_col desc limit 10 offset 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=10 OFFSET=5]
 |  order by: int_col DESC
 |
@@ -262,6 +296,8 @@ order by int_col desc limit 10 offset 5
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:MERGING-EXCHANGE [UNPARTITIONED]
 |  offset: 5
 |  order by: int_col DESC
@@ -285,6 +321,8 @@ select int_col, bigint_col from
    select * from functional.alltypessmall) t
 order by int_col desc limit 10 offset 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=10 OFFSET=5]
 |  order by: int_col DESC
 |
@@ -299,6 +337,8 @@ order by int_col desc limit 10 offset 5
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:MERGING-EXCHANGE [UNPARTITIONED]
 |  offset: 5
 |  order by: int_col DESC
@@ -327,6 +367,8 @@ order by int_col desc limit 10 offset 5
 select * from (select * from functional.alltypes limit 10) t
 order by int_col limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=10]
 |  order by: int_col ASC
 |
@@ -334,6 +376,8 @@ order by int_col limit 10
    partitions=24/24 files=24 size=478.45KB
    limit: 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=10]
 |  order by: int_col ASC
 |
@@ -351,6 +395,8 @@ select * from
    (select * from functional.alltypessmall) limit 10) t
 order by int_col limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=10]
 |  order by: int_col ASC
 |
@@ -363,6 +409,8 @@ order by int_col limit 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:TOP-N [LIMIT=10]
 |  order by: int_col ASC
 |


[5/7] incubator-impala git commit: IMPALA-2905: Handle coordinator fragment lifecycle like all others

Posted by he...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
index 0ef95b0..8c25730 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/analytic-fns.test
@@ -5,6 +5,8 @@ max(tinyint_col) over(partition by int_col)
 from functional.alltypes
 group by int_col, tinyint_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |
 03:ANALYTIC
@@ -32,6 +34,8 @@ max(int_col) over(partition by int_col, bool_col),
 max(int_col) over(partition by int_col, tinyint_col)
 from functional.alltypes
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |
 04:ANALYTIC
@@ -60,6 +64,8 @@ max(int_col) over(partition by int_col),
 min(int_col) over(order by int_col)
 from functional.alltypes
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:ANALYTIC
 |  functions: min(int_col)
 |  order by: int_col ASC
@@ -78,6 +84,8 @@ from functional.alltypes
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:ANALYTIC
 |  functions: min(int_col)
 |  order by: int_col ASC
@@ -115,6 +123,8 @@ max(int_col) over(partition by int_col order by bigint_col),
 max(int_col) over(partition by int_col order by bigint_col desc)
 from functional.alltypes
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:ANALYTIC
 |  functions: max(int_col)
 |  partition by: bool_col
@@ -155,6 +165,8 @@ from functional.alltypes
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:EXCHANGE [UNPARTITIONED]
 |
 08:ANALYTIC
@@ -226,6 +238,8 @@ min(int_col) over(partition by int_col, smallint_col order by bigint_col),
 max(int_col) over(partition by int_col, smallint_col order by int_col)
 from functional.alltypes
 ---- PLAN
+PLAN-ROOT SINK
+|
 11:ANALYTIC
 |  functions: min(int_col), max(int_col)
 |  order by: bigint_col ASC
@@ -276,6 +290,8 @@ from functional.alltypes
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:ANALYTIC
 |  functions: min(int_col), max(int_col)
 |  order by: bigint_col ASC
@@ -336,12 +352,16 @@ from functional.alltypes
 # basic analytic with default window and no partition/ordering
 select count(*) over() from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:ANALYTIC
 |  functions: count(*)
 |
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:ANALYTIC
 |  functions: count(*)
 |
@@ -354,6 +374,8 @@ select count(*) over() from functional.alltypesagg
 select tinyint_col, sum(bigint_col) over(partition by tinyint_col) sum_of_bigints
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:ANALYTIC
 |  functions: sum(bigint_col)
 |  partition by: tinyint_col
@@ -364,6 +386,8 @@ from functional.alltypesagg
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:ANALYTIC
@@ -381,6 +405,8 @@ from functional.alltypesagg
 # basic analytic with default window and ordering
 select int_col, rank() over(order by int_col) from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:ANALYTIC
 |  functions: rank()
 |  order by: int_col ASC
@@ -392,6 +418,8 @@ select int_col, rank() over(order by int_col) from functional.alltypesagg
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:ANALYTIC
 |  functions: rank()
 |  order by: int_col ASC
@@ -413,6 +441,8 @@ select bigint_col, count(double_col)
 from functional.alltypesagg
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:ANALYTIC
 |  functions: count(double_col)
 |  partition by: tinyint_col + 1, double_col / 2
@@ -426,6 +456,8 @@ limit 10
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |  limit: 10
 |
@@ -463,6 +495,8 @@ avg(double_col)
 from functional.alltypes
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:ANALYTIC
 |  functions: avg(double_col)
 |  order by: int_col DESC
@@ -490,6 +524,8 @@ limit 10
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:ANALYTIC
 |  functions: avg(double_col)
 |  order by: int_col DESC
@@ -528,6 +564,8 @@ select a.tinyint_col, a.int_col, count(a.double_col)
 from functional.alltypes a inner join functional.alltypessmall b on a.id = b.id
 order by a.tinyint_col, a.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:SORT
 |  order by: tinyint_col ASC, int_col ASC
 |
@@ -551,6 +589,8 @@ order by a.tinyint_col, a.int_col
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> a.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: tinyint_col ASC, int_col ASC
 |
@@ -596,6 +636,8 @@ from functional.alltypes
 group by 1
 order by 1, 2, 3
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:SORT
 |  order by: bool_col ASC, sum(min(int_col)) ASC, max(sum(bigint_col)) ASC
 |
@@ -624,6 +666,8 @@ order by 1, 2, 3
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: bool_col ASC, sum(min(int_col)) ASC, max(sum(bigint_col)) ASC
 |
@@ -700,6 +744,8 @@ min(int_col)
   rows between unbounded preceding and 2 following)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:ANALYTIC
 |  functions: count(double_col), last_value(double_col)
 |  partition by: tinyint_col, double_col
@@ -754,6 +800,8 @@ from functional.alltypesagg
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 12:EXCHANGE [UNPARTITIONED]
 |
 10:ANALYTIC
@@ -829,6 +877,8 @@ sum(smallint_col)
   rows between 2 preceding and 2 following)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:ANALYTIC
 |  functions: sum(tinyint_col)
 |  order by: int_col DESC
@@ -862,6 +912,8 @@ from functional.alltypesagg
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:ANALYTIC
 |  functions: sum(tinyint_col)
 |  order by: int_col DESC
@@ -912,6 +964,8 @@ select double_col, a, b, a + b, double_col + a from
      (select * from functional.alltypes) v1) v2
 order by 2, 3, 4
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: a ASC, b ASC, a + b ASC
 |
@@ -928,6 +982,8 @@ order by 2, 3, 4
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: a ASC, b ASC, a + b ASC
 |
@@ -959,6 +1015,8 @@ with v2 as
 select double_col, a, b, a + b, double_col + a from v2
 order by 2, 3, 4
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: a ASC, b ASC, a + b ASC
 |
@@ -975,6 +1033,8 @@ order by 2, 3, 4
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: a ASC, b ASC, a + b ASC
 |
@@ -1007,6 +1067,8 @@ select b from
    from functional.alltypes) v
 where e < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:SELECT
 |  predicates: count(1) < 10
 |
@@ -1029,6 +1091,8 @@ where e < 10
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 05:SELECT
@@ -1068,6 +1132,8 @@ union all
  from functional.alltypestiny)
 order by 1 desc nulls first
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: min(id) OVER(...) DESC NULLS FIRST
 |
@@ -1110,6 +1176,8 @@ order by 1 desc nulls first
 09:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: min(id) OVER(...) DESC NULLS FIRST
 |
@@ -1172,6 +1240,8 @@ where int_col in
   (select min(bigint_col) over(partition by bool_col)
    from functional.alltypestiny t2 where t2.id < 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: int_col = min(bigint_col)
 |  runtime filters: RF000 <- min(bigint_col)
@@ -1191,6 +1261,8 @@ where int_col in
    partitions=4/4 files=4 size=6.32KB
    runtime filters: RF000 -> int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [LEFT SEMI JOIN, BROADCAST]
@@ -1247,6 +1319,8 @@ where
   v.a != v.e and
   v.b != v.c
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SELECT
 |  predicates: min(int_col) < 1, max(int_col) < 2, bigint_col > 10, count(int_col) < 3, sum(int_col) < 4, avg(int_col) < 5, min(int_col) != count(int_col), min(int_col) != avg(int_col), max(int_col) != count(int_col), count(int_col) < bigint_col + 3, sum(int_col) < bigint_col + 4, min(int_col) < bigint_col + 1, max(int_col) < bigint_col + 2, avg(int_col) < bigint_col + 5
 |
@@ -1279,6 +1353,8 @@ where
    partitions=24/24 files=24 size=478.45KB
    predicates: int_col <= 10, int_col >= 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:SELECT
 |  predicates: min(int_col) < 1, max(int_col) < 2, bigint_col > 10, count(int_col) < 3, sum(int_col) < 4, avg(int_col) < 5, min(int_col) != count(int_col), min(int_col) != avg(int_col), max(int_col) != count(int_col), count(int_col) < bigint_col + 3, sum(int_col) < bigint_col + 4, min(int_col) < bigint_col + 1, max(int_col) < bigint_col + 2, avg(int_col) < bigint_col + 5
 |
@@ -1330,6 +1406,8 @@ inner join functional.alltypes t2
 on (t1.id = t2.id and t1.a = t2.int_col)
 where t2.id < 10 and t2.int_col < 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: id = t2.id, sum(int_col) = t2.int_col
 |
@@ -1362,6 +1440,8 @@ left outer join functional.alltypes t2
 on (t1.id = t2.id and t1.a = t2.int_col)
 where t2.id < 10 and t2.int_col < 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: id = t2.id, sum(int_col) = t2.int_col
 |  other predicates: t2.id < 10, t2.int_col < 20
@@ -1387,6 +1467,8 @@ select
 row_number() over(partition by tinyint_col order by id)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:ANALYTIC
 |  functions: row_number()
 |  partition by: tinyint_col
@@ -1410,6 +1492,8 @@ lead(int_col, 8, 20) over(partition by int_col order by id),
 lag(int_col, 8, 20) over(partition by int_col order by id)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:ANALYTIC
 |  functions: lag(int_col, 8, 20)
 |  partition by: int_col
@@ -1478,6 +1562,8 @@ count(bigint_col) over(partition by tinyint_col order by id
 #                       range between unbounded preceding and 10 following)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:ANALYTIC
 |  functions: count(bigint_col)
 |  partition by: tinyint_col
@@ -1544,6 +1630,8 @@ first_value(tinyint_col ignore nulls) over (order by id
                                             rows between 1 following and 2 following)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:ANALYTIC
 |  functions: last_value_ignore_nulls(tinyint_col)
 |  order by: id DESC
@@ -1607,6 +1695,8 @@ last_value(bigint_col) over(partition by tinyint_col order by id
                             range between unbounded preceding and unbounded following)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:ANALYTIC
 |  functions: last_value(bigint_col)
 |  partition by: tinyint_col
@@ -1654,6 +1744,8 @@ select DENSE_RANK() OVER (ORDER BY t1.day ASC)
 FROM functional.alltypesagg t1
 WHERE EXISTS (SELECT t1.year AS int_col_1 FROM functional.alltypesagg t1)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:ANALYTIC
 |  functions: dense_rank()
 |  order by: day ASC
@@ -1682,6 +1774,8 @@ FROM (
   FROM functional.alltypes t1 ) t1
 WHERE id IS NULL and tinyint_col != 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -1705,6 +1799,8 @@ WHERE id IS NULL and tinyint_col != 5
    partitions=4/4 files=4 size=460B
    predicates: t1.id IS NULL, t1.tinyint_col != 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |
@@ -1747,6 +1843,8 @@ select * from
    from functional.alltypes) v
 where year = 2009 and id = 1 and int_col < 10 and s = 4
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SELECT
 |  predicates: id = 1, int_col < 10, sum(int_col) = 4
 |
@@ -1775,6 +1873,8 @@ select * from
    from functional.alltypes) v
 where year = 2009 and tinyint_col + 1 = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:SELECT
 |  predicates: tinyint_col + 1 = 1
 |
@@ -1826,6 +1926,8 @@ select * from
    from functional.alltypes) v
 where year = 2009 and tinyint_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SELECT
 |  predicates: tinyint_col = 1, year = 2009
 |
@@ -1866,6 +1968,8 @@ select * from
    from functional.alltypestiny where id = tinyint_col) v
 where month = int_col and int_col = 1 and tinyint_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SELECT
 |  predicates: month = int_col
 |
@@ -1894,6 +1998,8 @@ select * from
    on t1.id = t2.id) v
 where v.x + v.y < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:SELECT
 |  predicates: tinyint_col + int_col < 10
 |
@@ -1928,6 +2034,8 @@ select * from
    from functional.alltypestiny) v
 where x = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SELECT
 |  predicates: int_col + 1 = 1
 |
@@ -1964,6 +2072,8 @@ left outer join
    from functional.alltypestiny) t2
 on (t1.id = t2.a + 100)
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:ANALYTIC
 |  functions: avg(if(TupleIsNull(), NULL, coalesce(id + bigint_col, 40)))
 |  order by: if(TupleIsNull(), NULL, coalesce(bigint_col, 30)) ASC
@@ -2023,6 +2133,8 @@ from
   on (t1.id = t2.a + 100)) t3
 group by d
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:AGGREGATE [FINALIZE]
 |  output: avg(sum(t1.id)), sum(avg(g)), count(id)
 |  group by: if(TupleIsNull(), NULL, coalesce(int_col, 20))
@@ -2090,6 +2202,8 @@ full outer join
    on (c.id = v3.id)) v4
 on (v2.id = v4.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 14:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: id = id
 |
@@ -2152,6 +2266,8 @@ left outer join
    on (t1.id = t2.id)) b
 on (a.id = b.id and b.int_col < 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.id = t1.id
 |
@@ -2198,6 +2314,8 @@ first_value(int_col) over (order by bigint_col
                            rows between unbounded preceding and current row)
 from functional.alltypes
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:ANALYTIC
 |  functions: last_value(int_col), first_value(int_col)
 |  order by: bigint_col ASC

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test b/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
index f0431a2..9c68c65 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/complex-types-file-formats.test
@@ -1,6 +1,8 @@
 # Scanning an unpartitioned Parquet table with complex types plans ok.
 select s.f1 from functional_parquet.complextypes_fileformat t, t.a
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [CROSS JOIN]
@@ -47,6 +49,8 @@ Complex types are supported for these file formats: PARQUET.
 # only scalar type columns are allowed.
 select id from functional_rc_snap.complextypes_fileformat
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional_rc_snap.complextypes_fileformat]
    partitions=1/1 files=1 size=56B
 ====
@@ -54,6 +58,8 @@ select id from functional_rc_snap.complextypes_fileformat
 # queries should work.
 select count(*) from functional_rc_snap.complextypes_fileformat
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -70,6 +76,8 @@ Complex types are supported for these file formats: PARQUET.
 # are allowed.
 select id from functional_seq_snap.complextypes_fileformat
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional_seq_snap.complextypes_fileformat]
    partitions=1/1 files=1 size=87B
 ====
@@ -85,6 +93,8 @@ not implemented: Scan of partition
 # columns are selected.
 select id from functional_hbase.allcomplextypes
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.allcomplextypes]
 ====
 # Scanning an HBase table with complex-types columns fails if a complex-typed
@@ -115,6 +125,8 @@ Complex types are supported for these file formats: PARQUET.
 # Scanning a Parquet partition of a multi-format table with complex types plans ok.
 select s.f1 from functional.complextypes_multifileformat t, t.a where p = 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [CROSS JOIN]
@@ -143,6 +155,8 @@ Complex types are supported for these file formats: PARQUET.
 # queries should work.
 select count(*) from functional.complextypes_multifileformat where p = 4
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/conjunct-ordering.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/conjunct-ordering.test b/testdata/workloads/functional-planner/queries/PlannerTest/conjunct-ordering.test
index 29e1864..9f97d62 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/conjunct-ordering.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/conjunct-ordering.test
@@ -4,6 +4,8 @@ from functional.alltypes a
 where a.int_col = a.tinyint_col and
       a.bool_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.bool_col, a.int_col = a.tinyint_col
@@ -14,6 +16,8 @@ from functional.alltypes a
 where a.string_col LIKE '%a%' and
       a.int_col = a.tinyint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.int_col = a.tinyint_col, a.string_col LIKE '%a%'
@@ -24,6 +28,8 @@ from functional.alltypes a
 where (a.int_col = a.tinyint_col or a.int_col = a.smallint_col) and
       a.int_col = a.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.int_col = a.bigint_col, (a.int_col = a.tinyint_col OR a.int_col = a.smallint_col)
@@ -33,6 +39,8 @@ select *
 from functional.alltypes a
 where a.int_col + 5 = a.bigint_col - 10 and a.int_col = a.tinyint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.int_col = a.tinyint_col, a.int_col + 5 = a.bigint_col - 10
@@ -43,6 +51,8 @@ from functional.alltypes a
 where a.int_col = a.tinyint_col and
       (case a.int_col when 0 then true when 1 then true when 2 then true else false end)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.int_col = a.tinyint_col, (CASE a.int_col WHEN 0 THEN TRUE WHEN 1 THEN TRUE WHEN 2 THEN TRUE ELSE FALSE END), (CASE a.tinyint_col WHEN 0 THEN TRUE WHEN 1 THEN TRUE WHEN 2 THEN TRUE ELSE FALSE END)
@@ -53,6 +63,8 @@ select *
 from functional.alltypes a
 where a.date_string_col LIKE 'a%a' and a.date_string_col LIKE '%a%'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.date_string_col LIKE '%a%', a.date_string_col LIKE 'a%a'
@@ -62,6 +74,8 @@ select *
 from functional.alltypes a
 where a.int_col IN (1, 2, 3, 4, 5, 6, 7, 8, 9) and a.int_col = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.int_col = 1, a.int_col IN (1, 2, 3, 4, 5, 6, 7, 8, 9)
@@ -71,6 +85,8 @@ select *
 from functional.alltypes a
 where a.timestamp_col > '2000-01-01' and a.int_col = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.int_col = 0, a.timestamp_col > '2000-01-01'
@@ -82,6 +98,8 @@ where a.string_col = "looooooooooooooooong string" and
       a.string_col = "medium string" and
       a.string_col = "a"
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.string_col = 'a', a.string_col = 'medium string', a.string_col = 'looooooooooooooooong string'
@@ -92,6 +110,8 @@ from functional.alltypes a
 where a.timestamp_col - interval 1 day > '2000-01-01' and
       a.timestamp_col < '2020-01-01'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.timestamp_col < '2020-01-01', a.timestamp_col - INTERVAL 1 day > '2000-01-01'
@@ -101,6 +121,8 @@ select *
 from functional.alltypes a
 where ceil(a.double_col) > 0 and a.double_col > 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.double_col > 0, ceil(a.double_col) > 0
@@ -110,6 +132,8 @@ select *
 from functional.alltypes a
 where cast(a.int_col as double) > 0 and a.int_col > 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.int_col > 0, CAST(a.int_col AS DOUBLE) > 0
@@ -119,6 +143,8 @@ select *
 from functional.alltypes a
 where a.string_col = "string" and a.int_col is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.int_col IS NULL, a.string_col = 'string'
@@ -131,6 +157,8 @@ where a.string_col LIKE '%a%' and
       (a.int_col = a.tinyint_col or a.int_col = a.smallint_col) and
       a.int_col = a.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.bool_col, a.int_col = a.bigint_col, (a.int_col = a.tinyint_col OR a.int_col = a.smallint_col), a.string_col LIKE '%a%'
@@ -141,6 +169,8 @@ select *
 from functional.alltypes a
 where a.int_col = 0 and a.id = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes a]
    partitions=24/24 files=24 size=478.45KB
    predicates: a.id = 0, a.int_col = 0

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/constant.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/constant.test b/testdata/workloads/functional-planner/queries/PlannerTest/constant.test
index 76b0fec..3c03cd5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/constant.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/constant.test
@@ -1,8 +1,12 @@
 select 1 + 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=1
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test b/testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
index 5662f5d..fd7dc1e 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/data-source-tables.test
@@ -5,6 +5,8 @@ where tinyint_col < 256 and
       float_col != 0 and
       cast(int_col as bigint) < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
 data source predicates: tinyint_col < 256
 predicates: float_col != 0, CAST(int_col AS BIGINT) < 10
@@ -22,6 +24,8 @@ where 10 > int_col and
       not true = bool_col and
       not 5.0 = double_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
 data source predicates: 10 > int_col, string_col != 'Foo'
 predicates: 5 > double_col, NOT 5.0 = double_col, NOT TRUE = bool_col, string_col != 'Bar'
@@ -34,6 +38,8 @@ where int_col < 10 and
       string_col in ("Foo", "Bar") and
       bool_col != false
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
 data source predicates: int_col < 10, bool_col != FALSE
 predicates: double_col > 5, string_col IN ('Foo', 'Bar')
@@ -50,6 +56,8 @@ and a.int_col = b.id and a.bigint_col = b.id
 # redundant predicates to test minimal spanning tree of equivalent slots
 where a.tinyint_col = a.smallint_col and a.int_col = a.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = b.id
 |
@@ -69,6 +77,8 @@ and smallint_col IS DISTINCT FROM 3
 and int_col is not distinct from 4
 and bigint_col is not distinct from 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN DATA SOURCE [functional.alltypes_datasource]
 data source predicates: id IS NOT DISTINCT FROM 1, tinyint_col IS DISTINCT FROM 2, int_col IS NOT DISTINCT FROM 4
 predicates: bigint_col IS NOT DISTINCT FROM 5, bool_col IS NOT DISTINCT FROM TRUE, smallint_col IS DISTINCT FROM 3

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/disable-preaggregations.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/disable-preaggregations.test b/testdata/workloads/functional-planner/queries/PlannerTest/disable-preaggregations.test
index f91dae5..07726c9 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/disable-preaggregations.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/disable-preaggregations.test
@@ -2,6 +2,8 @@ select tinyint_col, count(*)
 from functional.alltypesagg
 group by 1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -20,6 +22,8 @@ group by 1
 select count(distinct id)
 from functional.alltypesagg
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(id)
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test b/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test
index fca99ed..b895ad0 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/distinct-estimate.test
@@ -1,12 +1,16 @@
 # Distinct estimate
 select distinctpc(l_orderkey) from tpch.lineitem
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: distinctpc(l_orderkey)
 |
 00:SCAN HDFS [tpch.lineitem]
    partitions=1/1 files=1 size=718.94MB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: distinctpc:merge(l_orderkey)
 |
@@ -21,6 +25,8 @@ select distinctpc(l_orderkey) from tpch.lineitem
 # Distinct estimate with distinct
 select count(distinct l_orderkey), distinctpc(l_orderkey) from tpch.lineitem
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(l_orderkey), distinctpc:merge(l_orderkey)
 |
@@ -31,6 +37,8 @@ select count(distinct l_orderkey), distinctpc(l_orderkey) from tpch.lineitem
 00:SCAN HDFS [tpch.lineitem]
    partitions=1/1 files=1 size=718.94MB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(l_orderkey), distinctpc:merge(l_orderkey)
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test b/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test
index e5f3bce..b5361a6 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/distinct.test
@@ -2,12 +2,16 @@
 select distinct *
 from functional.testtbl
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: functional.testtbl.id, functional.testtbl.name, functional.testtbl.zip
 |
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -25,12 +29,16 @@ from functional.testtbl
 select distinct id, zip
 from functional.testtbl
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: id, zip
 |
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -48,6 +56,8 @@ from functional.testtbl
 select count(distinct id, zip)
 from functional.testtbl
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(if(id IS NULL, NULL, zip))
 |
@@ -57,6 +67,8 @@ from functional.testtbl
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(if(id IS NULL, NULL, zip))
 |
@@ -81,6 +93,8 @@ select tinyint_col, count(distinct int_col, bigint_col)
 from functional.alltypesagg
 group by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(if(int_col IS NULL, NULL, bigint_col))
 |  group by: tinyint_col
@@ -91,6 +105,8 @@ group by 1
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:AGGREGATE [FINALIZE]
@@ -113,6 +129,8 @@ select tinyint_col, count(distinct int_col), sum(distinct int_col)
 from functional.alltypesagg
 group by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(int_col), sum(int_col)
 |  group by: tinyint_col
@@ -123,6 +141,8 @@ group by 1
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:AGGREGATE [FINALIZE]
@@ -144,6 +164,8 @@ group by 1
 select sum(distinct int_col)
 from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: sum(int_col)
 |
@@ -153,6 +175,8 @@ from functional.alltypesagg
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum:merge(int_col)
 |
@@ -178,6 +202,8 @@ select tinyint_col, count(distinct int_col),
 min(distinct smallint_col), max(distinct string_col)
 from functional.alltypesagg group by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(int_col), min:merge(smallint_col), max:merge(string_col)
 |  group by: tinyint_col
@@ -189,6 +215,8 @@ from functional.alltypesagg group by 1
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:AGGREGATE [FINALIZE]
@@ -213,6 +241,8 @@ select tinyint_col, count(distinct int_col), count(*), sum(distinct int_col),
 sum(int_col), min(smallint_col), max(bigint_col)
 from functional.alltypesagg group by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(int_col), sum(int_col), count:merge(*), sum:merge(int_col), min:merge(smallint_col), max:merge(bigint_col)
 |  group by: tinyint_col
@@ -224,6 +254,8 @@ from functional.alltypesagg group by 1
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:AGGREGATE [FINALIZE]
@@ -250,6 +282,8 @@ select t1.c, t2.c from
 (select count(distinct int_col) as c from functional.alltypestiny) t1 inner join
 (select count(distinct bigint_col) as c from functional.alltypestiny) t2 on (t1.c = t2.c)
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: count(int_col) = count(bigint_col)
 |
@@ -271,6 +305,8 @@ select t1.c, t2.c from
 00:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: count(int_col) = count(bigint_col)
 |
@@ -323,6 +359,8 @@ select t1.c, t2.c from
 select count(distinct tinyint_col) from functional.alltypes
 having count(bigint_col) > 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(tinyint_col), count:merge(bigint_col)
 |  having: zeroifnull(count(bigint_col)) > 0
@@ -334,6 +372,8 @@ having count(bigint_col) > 0
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(tinyint_col), count:merge(bigint_col)
 |  having: zeroifnull(count(bigint_col)) > 0
@@ -362,6 +402,8 @@ select 1 from
   (select count(distinct 1) x from functional.alltypes) t
 where t.x is not null
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(1)
 |  having: count(1) IS NOT NULL
@@ -372,6 +414,8 @@ where t.x is not null
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(1)
 |  having: count(1) IS NOT NULL
@@ -398,6 +442,8 @@ select 1 from
   (select count(distinct 1) x, count(1) y from functional.alltypes) t
 where t.x + t.y > 10 and t.x > 0 and t.y > 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(1), count:merge(1)
 |  having: count(1) > 0, zeroifnull(count(1)) > 1, count(1) + zeroifnull(count(1)) > 10
@@ -409,6 +455,8 @@ where t.x + t.y > 10 and t.x > 0 and t.y > 1
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(1), count:merge(1)
 |  having: count(1) > 0, zeroifnull(count(1)) > 1, count(1) + zeroifnull(count(1)) > 10
@@ -434,6 +482,8 @@ where t.x + t.y > 10 and t.x > 0 and t.y > 1
 # IMPALA-2266: Test non-grouping distinct aggregation inside an inline view.
 select * from (select count(distinct int_col) cd from functional.alltypes) v
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(int_col)
 |
@@ -456,6 +506,8 @@ select * from (select count(distinct int_col) cd from functional.alltypes) v
 # IMPALA-2266: Test grouping distinct aggregation inside an inline view.
 select * from (select count(distinct int_col) cd from functional.alltypes group by bool_col) v
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:AGGREGATE [FINALIZE]
@@ -476,6 +528,8 @@ select * from (select count(distinct int_col) cd from functional.alltypes group
 # IMPALA-4042: count(distinct NULL) fails on a view
 select count(distinct null) from functional.alltypes_view
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count:merge(NULL)
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/empty.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/empty.test b/testdata/workloads/functional-planner/queries/PlannerTest/empty.test
index 27413c0..7933b3a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/empty.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/empty.test
@@ -5,11 +5,15 @@ left outer join functional.alltypes t2
 on t1.id = t2.id
 where false
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # HBase scan turns into empty-set node due to a constant conjunct.
 select * from functional_hbase.alltypessmall where false
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Data source scan turns into empty-set node due to a constant conjunct.
@@ -19,6 +23,8 @@ inner join functional.alltypestiny b
 on a.id = b.id
 where length("a") > 7
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Constant conjunct in ON clause turns query block into an empty-set node.
@@ -27,6 +33,8 @@ from functional.alltypestiny t1
 inner join functional.alltypes t2
 on (t1.id = t2.id and (false or false))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Constant conjunct in WHERE clause turns query block into an aggregation
@@ -35,6 +43,8 @@ select count(int_col), avg(double_col), count(*)
 from functional.alltypes
 where null
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(int_col), avg(double_col), count(*)
 |
@@ -47,6 +57,8 @@ from functional.alltypestiny t1
 inner join functional.alltypes t2
 on (t1.id = t2.id and (false or false))
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -61,6 +73,8 @@ on t1.id = t2.id
 group by t1.int_col
 having ifnull(null, false)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Constant conjunct causes empty-set inline view.
@@ -74,6 +88,8 @@ inner join
    where 1 + 3 > 10) e
 on e.id = f.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: f.id = t1.id
 |  runtime filters: RF000 <- t1.id
@@ -91,6 +107,8 @@ select * from functional.alltypes where "abc" = "cde"
 union all
 select * from functional.alltypestiny
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HDFS [functional.alltypestiny]
@@ -110,6 +128,8 @@ full outer join
    where null) t2
 on a.id = t2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: a.id = id
 |
@@ -125,6 +145,8 @@ from functional.alltypessmall a
 left outer join functional.alltypestiny b
 on (a.id = b.id and 1 + 1 > 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.id = b.id
 |  other join predicates: 1 + 1 > 10
@@ -142,6 +164,8 @@ from functional.alltypessmall a
 right outer join functional.alltypestiny b
 on (a.id = b.id and !true)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: a.id = b.id
 |  other join predicates: NOT TRUE
@@ -161,6 +185,8 @@ from functional.alltypessmall a
 full outer join functional.alltypestiny b
 on (a.id = b.id and null = "abc")
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: a.id = b.id
 |  other join predicates: NULL = 'abc'
@@ -178,6 +204,8 @@ left outer join functional.alltypes t2
 on t1.id = t2.id
 limit 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Limit 0 turns query block into an empty-set node.
@@ -185,6 +213,8 @@ select count(int_col), avg(double_col), count(*)
 from functional.alltypes
 limit 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Limit 0 causes empty-set inline view.
@@ -198,6 +228,8 @@ inner join
    limit 0) e
 on e.id = f.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: f.id = t1.id
 |  runtime filters: RF000 <- t1.id
@@ -215,6 +247,8 @@ select * from functional.alltypes limit 0
 union all
 select * from functional.alltypestiny
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HDFS [functional.alltypestiny]
@@ -231,6 +265,8 @@ union all
 (select * from functional.alltypestiny)
 limit 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # Inline view with a constant select stmt that is guaranteed to be empty.
@@ -241,6 +277,8 @@ w1 where w1.c1 is null
 union all
 select int_col from functional.alltypesagg
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--03:SCAN HDFS [functional.alltypesagg]
@@ -254,6 +292,8 @@ select int_col from functional.alltypesagg
 # IMPALA-1234: Analytic with constant empty result set failed precondition check in FE
 select MIN(int_col) OVER () FROM functional.alltypes limit 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # IMPALA-1860: INSERT/CTAS should evaluate and apply constant predicates.
@@ -291,6 +331,8 @@ from
   (select id, int_col, bigint_col from functional.alltypestiny) T
 where false
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: sum(id), count(int_col)
 |
@@ -304,6 +346,8 @@ from
   (select id, int_col from functional.alltypestiny) T2 on (T1.id = T2.id)
 where T1.bigint_col < 10 and 1 > 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: sum(id + int_col)
 |
@@ -317,6 +361,8 @@ from
   functional.alltypessmall T2 on T1.id = T2.id
 where T2.bigint_col < 10 and false
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: count(T1.int_col)
 |
@@ -338,6 +384,8 @@ union all
 select coalesce(10.4, int_col)
 from functional.alltypes where false
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:UNION
 |
 01:AGGREGATE [FINALIZE]
@@ -359,6 +407,8 @@ select 1
 union all select bigint_col
 from functional.alltypestiny
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:UNION
 |  constant-operands=1
 |
@@ -375,6 +425,8 @@ from functional.alltypestiny
 select * from (select 10 as i, 2 as j, '2013' as s) as t
 where t.i < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # IMPALA-2216: Make sure the final output exprs are substituted, even
@@ -406,6 +458,8 @@ left outer join
       where null) nv) v4
 where c_custkey < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--16:NESTED LOOP JOIN [LEFT OUTER JOIN]
@@ -457,6 +511,8 @@ from tpch_nested_parquet.customer c,
  ) v1
 where c_custkey = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--07:NESTED LOOP JOIN [CROSS JOIN]
@@ -479,5 +535,7 @@ where c_custkey = 1
 # IMPALA-2215: Having clause without aggregation.
 select 1 from (select 1) v having 1 > 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test b/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
index 57b2cce..91aff74 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/hbase.test
@@ -1,6 +1,8 @@
 # full scan of string typed row-key
 select * from functional_hbase.stringids
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
 ====
 # predicate on row key doesn't get transformed into scan parameter, because
@@ -8,6 +10,8 @@ select * from functional_hbase.stringids
 select * from functional_hbase.alltypessmall
 where id < 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    predicates: id < 5
 ---- SCANRANGELOCATIONS
@@ -16,6 +20,8 @@ NODE 0:
   HBASE KEYRANGE port=16202 3:7
   HBASE KEYRANGE port=16203 7:<unbounded>
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -26,6 +32,8 @@ select * from functional_hbase.stringids
 where id = '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 5
    stop key: 5\0
@@ -34,6 +42,8 @@ and tinyint_col = 5
 NODE 0:
   HBASE KEYRANGE port=16202 5:5\0
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -45,6 +55,8 @@ select * from functional_hbase.stringids
 where id > '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 5\0
    predicates: tinyint_col = 5
@@ -53,6 +65,8 @@ NODE 0:
   HBASE KEYRANGE port=16202 5\0:7
   HBASE KEYRANGE port=16203 7:<unbounded>
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -63,6 +77,8 @@ select * from functional_hbase.stringids
 where id >= '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 5
    predicates: tinyint_col = 5
@@ -71,6 +87,8 @@ NODE 0:
   HBASE KEYRANGE port=16202 5:7
   HBASE KEYRANGE port=16203 7:<unbounded>
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -81,6 +99,8 @@ select * from functional_hbase.stringids
 where id < '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    stop key: 5
    predicates: tinyint_col = 5
@@ -89,6 +109,8 @@ NODE 0:
   HBASE KEYRANGE port=16201 <unbounded>:3
   HBASE KEYRANGE port=16202 3:5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -99,6 +121,8 @@ select * from functional_hbase.stringids
 where id <= '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    stop key: 5\0
    predicates: tinyint_col = 5
@@ -107,6 +131,8 @@ select * from functional_hbase.stringids
 where id > '4' and id < '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 4\0
    stop key: 5
@@ -115,6 +141,8 @@ and tinyint_col = 5
 NODE 0:
   HBASE KEYRANGE port=16202 4\0:5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -126,6 +154,8 @@ select * from functional_hbase.stringids
 where id >= '4' and id < '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 4
    stop key: 5
@@ -134,6 +164,8 @@ and tinyint_col = 5
 NODE 0:
   HBASE KEYRANGE port=16202 4:5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -145,6 +177,8 @@ select * from functional_hbase.stringids
 where id > '4' and id <= '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 4\0
    stop key: 5\0
@@ -153,6 +187,8 @@ and tinyint_col = 5
 NODE 0:
   HBASE KEYRANGE port=16202 4\0:5\0
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -164,6 +200,8 @@ select * from functional_hbase.stringids
 where id >= '4' and id <= '5'
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 4
    stop key: 5\0
@@ -172,6 +210,8 @@ and tinyint_col = 5
 NODE 0:
   HBASE KEYRANGE port=16202 4:5\0
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -185,6 +225,8 @@ from functional_hbase.alltypessmall
 where id < 5
 group by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: int_col
@@ -192,6 +234,8 @@ group by 1
 00:SCAN HBASE [functional_hbase.alltypessmall]
    predicates: id < 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -210,10 +254,14 @@ group by 1
 # predicates on string columns against a constant string are converted to HBase filters
 select * from functional_hbase.alltypessmall where string_col = '4'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters: d:string_col EQUAL '4'
    predicates: string_col = '4'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -223,10 +271,14 @@ select * from functional_hbase.alltypessmall where string_col = '4'
 # test all comparison ops
 select * from functional_hbase.alltypessmall where string_col != '4'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters: d:string_col NOT_EQUAL '4'
    predicates: string_col != '4'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -235,10 +287,14 @@ select * from functional_hbase.alltypessmall where string_col != '4'
 ====
 select * from functional_hbase.alltypessmall where string_col < '4'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters: d:string_col LESS '4'
    predicates: string_col < '4'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -247,10 +303,14 @@ select * from functional_hbase.alltypessmall where string_col < '4'
 ====
 select * from functional_hbase.alltypessmall where string_col > '4'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters: d:string_col GREATER '4'
    predicates: string_col > '4'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -259,10 +319,14 @@ select * from functional_hbase.alltypessmall where string_col > '4'
 ====
 select * from functional_hbase.alltypessmall where string_col <= '4'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters: d:string_col LESS_OR_EQUAL '4'
    predicates: string_col <= '4'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -271,10 +335,14 @@ select * from functional_hbase.alltypessmall where string_col <= '4'
 ====
 select * from functional_hbase.alltypessmall where string_col >= '4'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters: d:string_col GREATER_OR_EQUAL '4'
    predicates: string_col >= '4'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -285,6 +353,8 @@ select * from functional_hbase.alltypessmall where string_col >= '4'
 select * from functional_hbase.alltypessmall
 where string_col >= '4' and string_col != '2' and date_string_col = '04/03/09'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters:
   d:string_col NOT_EQUAL '2'
@@ -292,6 +362,8 @@ where string_col >= '4' and string_col != '2' and date_string_col = '04/03/09'
   d:date_string_col EQUAL '04/03/09'
    predicates: string_col != '2', string_col >= '4', date_string_col = '04/03/09'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -304,6 +376,8 @@ where string_col >= '4' and string_col != '2' and date_string_col = '04/03/09'
 # mix of predicates and functional_hbase. filters
 select * from functional_hbase.alltypessmall where string_col = '4' and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    hbase filters: d:string_col EQUAL '4'
    predicates: tinyint_col = 5, string_col = '4'
@@ -312,6 +386,8 @@ select * from functional_hbase.alltypessmall where string_col = '4' and tinyint_
 select * from functional_hbase.stringids
 where string_col = '4' and tinyint_col = 5 and id >= '4' and id <= '5'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 4
    stop key: 5\0
@@ -321,6 +397,8 @@ where string_col = '4' and tinyint_col = 5 and id >= '4' and id <= '5'
 NODE 0:
   HBASE KEYRANGE port=16202 4:5\0
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -332,12 +410,16 @@ NODE 0:
 # predicates involving casts (ie, non-string comparisons) cannot be turned into filters
 select * from functional_hbase.alltypessmall where cast(string_col as int) >= 4
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    predicates: CAST(string_col AS INT) >= 4
 ====
 # non-const comparisons cannot be turned into filters
 select * from functional_hbase.alltypessmall where string_col >= date_string_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    predicates: string_col >= date_string_col
 ====
@@ -346,6 +428,8 @@ select * from functional_hbase.stringids
 where id = concat('', '5')
 and tinyint_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 5
    stop key: 5\0
@@ -356,6 +440,8 @@ select * from functional_hbase.stringids
 where string_col = '4' and tinyint_col = 5
   and id >= concat('', '4') and id <= concat('5', '')
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    start key: 4
    stop key: 5\0
@@ -365,6 +451,8 @@ where string_col = '4' and tinyint_col = 5
 NODE 0:
   HBASE KEYRANGE port=16202 4:5\0
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.stringids]
@@ -376,8 +464,12 @@ NODE 0:
 # IMP-1188 - row key predicate is null.
 select * from functional_hbase.stringids where id = null
 ---- PLAN
+PLAN-ROOT SINK
+|
 empty scan node
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 empty scan node
@@ -385,8 +477,12 @@ empty scan node
 # IMP-1188 - row key lower bound is bigger than upper bound.
 select * from functional_hbase.stringids where id > 'b' and id < 'a'
 ---- PLAN
+PLAN-ROOT SINK
+|
 empty scan node
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 empty scan node
@@ -396,6 +492,8 @@ empty scan node
 select * from functional_hbase.stringids
 where cast(id as int) < 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.stringids]
    predicates: CAST(id AS INT) < 5
 ====
@@ -407,6 +505,8 @@ where cast(id as int) < 5
 select * from functional_hbase.alltypesagg
 where bigint_col is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypesagg]
    predicates: bigint_col IS NULL
 ====
@@ -415,6 +515,8 @@ where bigint_col is null
 select bigint_col, day from functional_hbase.alltypesagg
 where bigint_col is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypesagg]
    predicates: bigint_col IS NULL
 ====
@@ -422,6 +524,8 @@ where bigint_col is null
 select * from functional_hbase.alltypesagg
 where bigint_col is not null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypesagg]
    predicates: bigint_col IS NOT NULL
 ====
@@ -429,6 +533,8 @@ where bigint_col is not null
 select * from functional_hbase.alltypesagg
 where bigint_col is null and day = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypesagg]
    predicates: bigint_col IS NULL, day = 1
 ====
@@ -436,6 +542,8 @@ where bigint_col is null and day = 1
 select * from functional_hbase.alltypesagg
 where bigint_col is not null and bool_col = true
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypesagg]
    predicates: bigint_col IS NOT NULL, bool_col = TRUE
 ---- SCANRANGELOCATIONS
@@ -444,6 +552,8 @@ NODE 0:
   HBASE KEYRANGE port=16202 3:7
   HBASE KEYRANGE port=16203 7:<unbounded>
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypesagg]
@@ -453,12 +563,16 @@ NODE 0:
 select count(*) from functional_hbase.alltypesagg
 where bigint_col = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
 00:SCAN HBASE [functional_hbase.alltypesagg]
    predicates: bigint_col = 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |
@@ -474,6 +588,8 @@ where bigint_col = 10
 select count(*) from functional_hbase.alltypesagg
 where bigint_col = 10 and day = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -492,6 +608,8 @@ where
   a.int_col = b.int_col and
   c.int_col = b.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: b.int_col = a.int_col
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test b/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test
index 785d111..9d43a25 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/hdfs.test
@@ -2,9 +2,13 @@
 select * FROM functional.alltypes
 where cast(year as string) = to_date( from_unixtime(unix_timestamp()) )
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=0/24 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -16,6 +20,8 @@ from functional.testtbl
 where name like 'm%'
 group by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |  group by: zip
@@ -24,6 +30,8 @@ group by 1
    partitions=1/1 files=0 size=0B
    predicates: name LIKE 'm%'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -43,6 +51,8 @@ group by 1
 # all partitions are selected
 select * from functional.alltypes
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- SCANRANGELOCATIONS
@@ -72,6 +82,8 @@ NODE 0:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypes/year=2010/month=8/100801.txt 0:20853
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypes/year=2010/month=9/100901.txt 0:20179
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -80,6 +92,8 @@ NODE 0:
 # predicate on first partition key
 select id, month from functional.alltypes where year = 2009
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=238.68KB
 ---- SCANRANGELOCATIONS
@@ -97,6 +111,8 @@ NODE 0:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypes/year=2009/month=8/090801.txt 0:20853
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypes/year=2009/month=9/090901.txt 0:20179
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -105,141 +121,191 @@ NODE 0:
 # same predicate, phrased differently
 select * from functional.alltypes where year = 2009.0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=238.68KB
 ====
 select * from functional.alltypes where 2009 = year
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=238.68KB
 ====
 select * from functional.alltypes where 2009 <=> year
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=238.68KB
 ====
 # compound predicate on the second partition key
 select * from functional.alltypes where !(month > 2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=4/24 files=4 size=76.83KB
 ====
 # nested compound predicates on the second partition key
 select * from functional.alltypes where !(!(month=1))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=2/24 files=2 size=40.32KB
 ====
 select * from functional.alltypes where !(!(month<=>1))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=2/24 files=2 size=40.32KB
 ====
 # predicates on both partition keys one of which is a compound predicate with NOT
 select * from functional.alltypes where year=2009 and !(month < 6)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=7/24 files=7 size=140.58KB
 ====
 # compound predicates on both partition keys
 select * from functional.alltypes where !(year < 2009) and !(month < 6)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=14/24 files=14 size=281.15KB
 ====
 # compound predicate on a conjunct
 select * from functional.alltypes where !(year = 2009 and month > 6)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=18/24 files=18 size=357.58KB
 ====
 select * from functional.alltypes where !(year <=> 2009 and month > 6)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=18/24 files=18 size=357.58KB
 ====
 select * from functional.alltypes where !(year <=> 2009) or !(month > 6)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=18/24 files=18 size=357.58KB
 ====
 # compound predicate on a disjunct
 select * from functional.alltypes where !(month = 6 or month = 8)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=20/24 files=20 size=398.31KB
 ====
 select * from functional.alltypes where !(month <=> 6 or month <=> 8)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=20/24 files=20 size=398.31KB
 ====
 # not predicate with is null
 select * from functional.alltypes where not (year = 2009 or month is null)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=239.77KB
 ====
 # not predicate with "<=> null" as a synonym of "is null"
 select * from functional.alltypes where not (year = 2009 or month <=> null)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=239.77KB
 ====
 # nested not predicates with is null
 select * from functional.alltypes where not (not (month is null))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=0/24 files=0 size=0B
 ====
 # nested not predicates with "<=> null" as a synonym of "is null"
 select * from functional.alltypes where not (not (month <=> null))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=0/24 files=0 size=0B
 ====
 # nested not predicates with disjunct
 select * from functional.alltypes where not (not (month is null or year = 2009))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=238.68KB
 ====
 # nested not predicates with disjunct and "<=> null" as a synonym of "is null"
 select * from functional.alltypes where not (not (month <=> null or year = 2009))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=238.68KB
 ====
 # predicate on second partition key
 select * from functional.alltypes where month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=2/24 files=2 size=40.32KB
 ====
 # predicate on both partition keys
 select * from functional.alltypes where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=1/24 files=1 size=19.95KB
 ====
 # single-sided range on 2nd key
 select * from functional.alltypes where year=2009 and month > 6
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=6/24 files=6 size=120.87KB
 ====
 select * from functional.alltypes where year=2009 and month < 6
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=5/24 files=5 size=98.11KB
 ====
 select * from functional.alltypes where year=2009 and month in (1, 3, 5, 7)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=4/24 files=4 size=80.74KB
 ====
 select * from functional.alltypes where year<=>2009 and month in (1, 3, 5, 7)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=4/24 files=4 size=80.74KB
 ====
@@ -247,6 +313,8 @@ select * from functional.alltypes where year<=>2009 and month in (1, 3, 5, 7)
 select * from functional.alltypes
 where year=2009 and month in (1, 3, 5, 7) and month is not null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=4/24 files=4 size=80.74KB
 ====
@@ -254,29 +322,39 @@ where year=2009 and month in (1, 3, 5, 7) and month is not null
 select * from functional.alltypes
 where year=2009 and month in (1, 3, 5, 7) and month is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=0/24 files=0 size=0B
 ====
 select * from functional.alltypes where year=2009 and (month in (1, 3, 5) or month = 7)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=4/24 files=4 size=80.74KB
 ====
 # single-sided ranges on both keys
 select * from functional.alltypes where year<=2009 and month < 6
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=5/24 files=5 size=98.11KB
 ====
 # range on 2nd key
 select * from functional.alltypes where month < 9 and month > 6
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=4/24 files=4 size=81.46KB
 ====
 # multiple predicates on first key; 2nd one applied as predicate
 select * from functional.alltypes where year < 2010 and year < 2009 and month > 6
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=0/24 files=0 size=0B
 ====
@@ -284,6 +362,8 @@ select * from functional.alltypes where year < 2010 and year < 2009 and month >
 select * from functional.alltypes
 where year < 2010 and (month > 6 or month = 1 or month in (3, 4))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=9/24 files=9 size=180.49KB
 ====
@@ -291,18 +371,24 @@ where year < 2010 and (month > 6 or month = 1 or month in (3, 4))
 select * from functional.alltypes
 where year < 2010 and (month > 6 or month <=> 1 or month in (3, 4))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=9/24 files=9 size=180.49KB
 ====
 # between predicate on second key
 select * from functional.alltypes where year = 2009 and month between 6 and 8
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=3/24 files=3 size=60.43KB
 ====
 # between predicate on second key
 select * from functional.alltypes where year <=> 2009 and month between 6 and 8
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=3/24 files=3 size=60.43KB
 ====
@@ -310,6 +396,8 @@ select * from functional.alltypes where year <=> 2009 and month between 6 and 8
 select * from functional.alltypes
 where year between 2009 and 2009 and month between 6 and 8
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=3/24 files=3 size=60.43KB
 ====
@@ -317,6 +405,8 @@ where year between 2009 and 2009 and month between 6 and 8
 select * from functional.alltypes
 where year = 2009 and (month between 6 and 7 or month between 7 and 8)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=3/24 files=3 size=60.43KB
 ---- SCANRANGELOCATIONS
@@ -329,6 +419,8 @@ NODE 0:
 select * from functional.alltypes
 where year = 2009 and (month between 5+1 and 8-1 or month between 9-2 and 1+7)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=3/24 files=3 size=60.43KB
 ---- SCANRANGELOCATIONS
@@ -340,11 +432,15 @@ NODE 0:
 # slot binding still determined
 select * from functional.alltypes where year - 1 = 2009
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=239.77KB
 ====
 select * from functional.alltypes where year - 1 <=> 2009
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=12/24 files=12 size=239.77KB
 ====
@@ -352,82 +448,110 @@ select * from functional.alltypes where year - 1 <=> 2009
 # IS NULL predicate on a partition key with nulls
 select * from functional.alltypesagg where day is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # <=> null predicate on a partition key with nulls
 select * from functional.alltypesagg where day <=> null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # IS NOT NULL predicate on a partition key with nulls
 select * from functional.alltypesagg where day is not null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=10/11 files=10 size=743.67KB
 ====
 # IS DISTINCT FROM NULL predicate on a partition key with nulls
 select * from functional.alltypesagg where day is distinct from null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=10/11 files=10 size=743.67KB
 ====
 select * from functional.alltypesagg where day = day
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=10/11 files=10 size=743.67KB
 ====
 select * from functional.alltypesagg where day <=> day
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ====
 # partition key predicates which are in conjunctive normal form (case 1)
 select * from functional.alltypesagg where day is null and day = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=0/11 files=0 size=0B
 ====
 # partition key predicates which are in conjunctive normal form (case 1)
 select * from functional.alltypesagg where day <=> null and day = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=0/11 files=0 size=0B
 ====
 # partition key predicates which are in conjunctive normal form (case 2)
 select * from functional.alltypesagg where day is null and month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # partition key predicates which are in conjunctive normal form (case 2)
 select * from functional.alltypesagg where day <=> null and month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # partition key predicates which are in conjunctive normal form (case 3)
 select * from functional.alltypesagg where month = 1 and (day is null or day = 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=2/11 files=2 size=145.53KB
 ====
 # partition key predicates which are in conjunctive normal form (case 3)
 select * from functional.alltypesagg where month = 1 and (day <=> null or day = 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=2/11 files=2 size=145.53KB
 ====
 # partition key predicates which are in conjunctive normal form (case 4)
 select * from functional.alltypesagg where month = 1 and (day is null or year = 2010)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ====
 # partition key predicates which are in conjunctive normal form (case 4)
 select * from functional.alltypesagg where month = 1 and (day <=> null or year = 2010)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ====
@@ -435,6 +559,8 @@ select * from functional.alltypesagg where month = 1 and (day <=> null or year =
 select * from functional.alltypesagg
 where (year = 2010 or month = 1) and (day is not null or day = 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=10/11 files=10 size=743.67KB
 ====
@@ -442,42 +568,56 @@ where (year = 2010 or month = 1) and (day is not null or day = 10)
 select * from functional.alltypesagg
 where (year = 2010 or month = 1) and (day is distinct from null or day = 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=10/11 files=10 size=743.67KB
 ====
 # partition key predicates which are in disjunctive normal form (case 1)
 select * from functional.alltypesagg where day is null or month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ====
 # partition key predicates which are in disjunctive normal form (case 1)
 select * from functional.alltypesagg where day <=> null or month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ====
 # partition key predicates which are in disjunctive normal form (case 2)
 select * from functional.alltypesagg where day is null or day = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=2/11 files=2 size=145.53KB
 ====
 # partition key predicates which are in disjunctive normal form (case 2)
 select * from functional.alltypesagg where day <=> null or day = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=2/11 files=2 size=145.53KB
 ====
 # partition key predicates which are in disjunctive normal form (case 3)
 select * from functional.alltypesagg where day = 10 or (day is null and year = 2010)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=2/11 files=2 size=145.53KB
 ====
 # partition key predicates which are in disjunctive normal form (case 3)
 select * from functional.alltypesagg where day = 10 or (day <=> null and year = 2010)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=2/11 files=2 size=145.53KB
 ====
@@ -485,6 +625,8 @@ select * from functional.alltypesagg where day = 10 or (day <=> null and year =
 select * from functional.alltypesagg
 where (month = 1 and day = 1) or (day is null and year = 2010)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=2/11 files=2 size=144.45KB
 ====
@@ -492,54 +634,72 @@ where (month = 1 and day = 1) or (day is null and year = 2010)
 select * from functional.alltypesagg
 where (month = 1 and day = 1) or (day <=> null and year = 2010)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=2/11 files=2 size=144.45KB
 ====
 # partition key predicates with negation (case 1)
 select * from functional.alltypesagg where not (day is not null)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # partition key predicates with negation (case 1)
 select * from functional.alltypesagg where not (day is distinct from null)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # partition key predicates with negation (case 2)
 select * from functional.alltypesagg where not (not (day is null))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # partition key predicates with negation (case 2)
 select * from functional.alltypesagg where not (not (day <=> null))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # partition key predicates with negation (case 3)
 select * from functional.alltypesagg where not (day is not null and month = 1)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # partition key predicates with negation (case 3)
 select * from functional.alltypesagg where not (day is distinct from null and month = 1)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
 # partition key predicates with negation (case 3)
 select * from functional.alltypesagg where not (day is not null or day < 9)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=0/11 files=0 size=0B
 ====
 # partition key predicates with negation (case 3)
 select * from functional.alltypesagg where not (day is distinct from null or day < 9)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=0/11 files=0 size=0B
 ====
@@ -547,6 +707,8 @@ select * from functional.alltypesagg where not (day is distinct from null or day
 select * from functional.alltypesagg
 where not (day is not null and (not (day < 9 and month = 1)))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=9/11 files=9 size=665.77KB
 ====
@@ -554,6 +716,8 @@ where not (day is not null and (not (day < 9 and month = 1)))
 select * from functional.alltypesagg
 where not (day is distinct from null and (not (day < 9 and month = 1)))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=9/11 files=9 size=665.77KB
 ====
@@ -561,6 +725,8 @@ where not (day is distinct from null and (not (day < 9 and month = 1)))
 select * from functional.alltypesagg
 where not (day is not null or (day = 1 and (not (month = 1 or year = 2010))))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
@@ -568,6 +734,8 @@ where not (day is not null or (day = 1 and (not (month = 1 or year = 2010))))
 select * from functional.alltypesagg
 where not (day is distinct from null or (day = 1 and (not (month = 1 or year = 2010))))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
@@ -575,6 +743,8 @@ where not (day is distinct from null or (day = 1 and (not (month = 1 or year = 2
 select * from functional.alltypesagg
 where year + 1 = 2011 and month + 1 <= 3 and day is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
@@ -582,6 +752,8 @@ where year + 1 = 2011 and month + 1 <= 3 and day is null
 select * from functional.alltypesagg
 where year + 1 = 2011 and month + 1 <= 3 and day <=> null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=1/11 files=1 size=71.05KB
 ====
@@ -591,6 +763,8 @@ where day = 5 or (day >= 1 and day <= 2) or (day > 6 and day < 8)
 or day is null or day in (4) or not(day is not null)
 or not (day not in (10)) or not (day != 8)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=8/11 files=8 size=591.30KB
 ---- SCANRANGELOCATIONS
@@ -610,6 +784,8 @@ where day = 5 or (day >= 1 and day <= 2) or (day > 6 and day < 8)
 or day <=> null or day in (4) or not(day is distinct from null)
 or not (day not in (10)) or not (day != 8)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg]
    partitions=8/11 files=8 size=591.30KB
 ---- SCANRANGELOCATIONS
@@ -626,47 +802,65 @@ NODE 0:
 # Predicates on a partition key with no values (see IMPALA-4128).
 select * from functional.emptytable where f2 = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.emptytable]
    partitions=0/0 files=0 size=0B
 ====
 select * from functional.emptytable where f2 != 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.emptytable]
    partitions=0/0 files=0 size=0B
 ====
 select * from functional.emptytable where f2 > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.emptytable]
    partitions=0/0 files=0 size=0B
 ====
 select * from functional.emptytable where f2 < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.emptytable]
    partitions=0/0 files=0 size=0B
 ====
 select * from functional.emptytable where f2 in (10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.emptytable]
    partitions=0/0 files=0 size=0B
 ====
 select * from functional.emptytable where f2 not in (10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.emptytable]
    partitions=0/0 files=0 size=0B
 ====
 select * from functional.emptytable where f2 is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.emptytable]
    partitions=0/0 files=0 size=0B
 ====
 select * from functional.emptytable where f2 is not null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.emptytable]
    partitions=0/0 files=0 size=0B
 ====
 # multi-file non-partitioned table
 select * from functional.alltypesaggmultifilesNoPart
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesaggmultifilesnopart]
    partitions=1/1 files=4 size=805.23KB
 ---- SCANRANGELOCATIONS
@@ -679,6 +873,8 @@ NODE 0:
 # multi-file partitioned table
 select * from functional.alltypesaggmultifiles where day <= 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesaggmultifiles]
    partitions=2/11 files=8 size=145.97KB
 ====
@@ -688,39 +884,53 @@ select * from functional.alltypesaggmultifiles where day <= 2
 # Test single binary predicate on a partition column
 select * from scale_db.num_partitions_1234_blocks_per_partition_1 where j = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [scale_db.num_partitions_1234_blocks_per_partition_1]
    partitions=1/1234 files=1 size=2B
 ====
 select * from scale_db.num_partitions_1234_blocks_per_partition_1 where j <=> 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [scale_db.num_partitions_1234_blocks_per_partition_1]
    partitions=1/1234 files=1 size=2B
 ====
 # Test disjunctive predicate on a partition column
 select * from scale_db.num_partitions_1234_blocks_per_partition_1 where j = 1 or j = 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [scale_db.num_partitions_1234_blocks_per_partition_1]
    partitions=2/1234 files=2 size=4B
 ====
 select * from scale_db.num_partitions_1234_blocks_per_partition_1 where j <=> 1 or j <=> 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [scale_db.num_partitions_1234_blocks_per_partition_1]
    partitions=2/1234 files=2 size=4B
 ====
 # Test conjunctive predicate on a partition column
 select * from scale_db.num_partitions_1234_blocks_per_partition_1 where j = 1 and j = 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [scale_db.num_partitions_1234_blocks_per_partition_1]
    partitions=0/1234 files=0 size=0B
 ====
 select * from scale_db.num_partitions_1234_blocks_per_partition_1 where j <=> 1 and j <=> 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [scale_db.num_partitions_1234_blocks_per_partition_1]
    partitions=0/1234 files=0 size=0B
 ====
 # Partition pruning when a binary predicate contains a NullLiteral (IMPALA-1535)
 select * from functional.alltypestiny t1 where t1.year != null or t1.year = null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypestiny t1]
    partitions=0/4 files=0 size=0B
 ====
@@ -728,6 +938,8 @@ select * from functional.alltypestiny t1 where t1.year != null or t1.year = null
 select * from functional.alltypestiny t1
 where t1.year IS DISTINCT FROM null or t1.year = null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypestiny t1]
    partitions=4/4 files=4 size=460B
 ====
@@ -735,6 +947,8 @@ where t1.year IS DISTINCT FROM null or t1.year = null
 # expression
 select * from functional.alltypesagg t1 where t1.year + null != t1.day
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg t1]
    partitions=0/11 files=0 size=0B
 ====
@@ -742,6 +956,8 @@ select * from functional.alltypesagg t1 where t1.year + null != t1.day
 # expression and IS DISTINCT FROM
 select * from functional.alltypesagg t1 where t1.year + null IS DISTINCT FROM t1.day
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg t1]
    partitions=10/11 files=10 size=743.67KB
 ====
@@ -749,6 +965,8 @@ select * from functional.alltypesagg t1 where t1.year + null IS DISTINCT FROM t1
 # (a single partition is scanned)
 select * from functional.alltypesagg t1 where day in (10, null)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg t1]
    partitions=1/11 files=1 size=74.48KB
 ====
@@ -756,6 +974,8 @@ select * from functional.alltypesagg t1 where day in (10, null)
 # (all partitions are pruned)
 select * from functional.alltypesagg t1 where day not in (10, null)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg t1]
    partitions=0/11 files=0 size=0B
 ====
@@ -763,6 +983,8 @@ select * from functional.alltypesagg t1 where day not in (10, null)
 select * from functional.alltypesagg t1
 where t1.day = instr("this is a test", "this") or t1.year = year(now()) + 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg t1]
    partitions=1/11 files=1 size=73.39KB
 ====
@@ -771,6 +993,8 @@ where t1.day = instr("this is a test", "this") or t1.year = year(now()) + 100
 select * from functional.alltypesagg t1
 where t1.day in (1, cast(2.0 as INT), year(now()) + 100)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg t1]
    partitions=2/11 files=2 size=147.87KB
 ====
@@ -779,6 +1003,8 @@ where t1.day in (1, cast(2.0 as INT), year(now()) + 100)
 select * from functional.alltypesagg t1
 where -t1.day in(-1 - 1) or cast(t1.day as string) like '%1%'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypesagg t1]
    partitions=3/11 files=3 size=222.34KB
 ====

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/implicit-joins.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/implicit-joins.test b/testdata/workloads/functional-planner/queries/PlannerTest/implicit-joins.test
index 8fd3ce2..229c1e6 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/implicit-joins.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/implicit-joins.test
@@ -5,6 +5,8 @@ from (select * from functional.alltypestiny) t1
   join (select * from functional.alltypestiny) t2 on (t1.id = t2.id)
   join functional.alltypestiny t3 on (coalesce(t1.id, t3.id) = t3.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: (coalesce(functional.alltypestiny.id, t3.id) = t3.id)
 |
@@ -30,6 +32,8 @@ from
 where
   v.x < v.y
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: t1.id < t2.id
 |
@@ -47,6 +51,8 @@ from
 where
   t1.id = t2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--02:SCAN HDFS [functional.alltypes t3]
@@ -71,6 +77,8 @@ from
 where
   t1.id = t3.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--01:SCAN HDFS [functional.alltypes t2]
@@ -98,6 +106,8 @@ from
 where
   t2.id = t3.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t2.id = t3.id
 |  runtime filters: RF000 <- t3.id
@@ -121,6 +131,8 @@ from
   functional.alltypes t3 on t3.id = t2.id,
   functional.alltypes t4
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--03:SCAN HDFS [functional.alltypes t4]
@@ -147,6 +159,8 @@ select a.* from
   functional.alltypessmall b full outer join
   functional.alltypes c on a.id = c.id and a.id < b.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: a.id = c.id
 |  other join predicates: a.id < b.id
@@ -169,6 +183,8 @@ functional.alltypestiny c,
 functional.alltypes d
 where a.id = d.id and b.id = c.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: d.id = a.id
 |  runtime filters: RF000 <- a.id
@@ -201,6 +217,8 @@ from
     on t1.id < t2.id,
   functional.alltypes t3
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -232,6 +250,8 @@ from
 where
   t1.id = t2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -267,6 +287,8 @@ from
 where
   t1.id = t2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -298,6 +320,8 @@ from
   functional.alltypes t0 left semi join
   functional.alltypes t1 on ( t0.id < t1.id )
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -320,6 +344,8 @@ where
   b.id in (select avg(id) from functional.alltypes group by month) and
   a.id < b.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -348,6 +374,8 @@ from functional.alltypes t1,
 functional.alltypes t2 join functional.alltypes t3 on (t1.id = t2.id),
 functional.alltypes t4
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--03:SCAN HDFS [functional.alltypes t4]



[4/7] incubator-impala git commit: IMPALA-2905: Handle coordinator fragment lifecycle like all others

Posted by he...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test b/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
index 79f75b6..556ba65 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/inline-view-limit.test
@@ -1,10 +1,14 @@
 # predicate pushdown
 select * from (select * from functional.alltypessmall) a where id < 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
    predicates: functional.alltypessmall.id < 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypessmall]
@@ -14,6 +18,8 @@ select * from (select * from functional.alltypessmall) a where id < 5
 # predicate pushdown is prevented in presence of limit clause
 select * from (select * from functional.alltypessmall limit 10) a where id < 5 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SELECT
 |  predicates: functional.alltypessmall.id < 5
 |  limit: 5
@@ -22,6 +28,8 @@ select * from (select * from functional.alltypessmall limit 10) a where id < 5 l
    partitions=4/4 files=4 size=6.32KB
    limit: 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:SELECT
 |  predicates: functional.alltypessmall.id < 5
 |  limit: 5
@@ -39,6 +47,8 @@ select *
 from (select * from functional.alltypessmall order by id limit 10) a
 where id < 5 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SELECT
 |  predicates: id < 5
 |  limit: 5
@@ -49,6 +59,8 @@ where id < 5 limit 5
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:SELECT
 |  predicates: id < 5
 |  limit: 5
@@ -72,6 +84,8 @@ from functional.alltypes
     group by 1 order by 2 limit 5) a using (id)
 where a.id < 5 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypes.id = id
 |  runtime filters: RF000 <- id
@@ -92,6 +106,8 @@ where a.id < 5 limit 5
    predicates: functional.alltypes.id < 5
    runtime filters: RF000 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |  limit: 5
 |
@@ -135,6 +151,8 @@ from (
   limit 10) a
 where id < 5 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:SELECT
 |  predicates: a.id < 5
 |  limit: 5
@@ -151,6 +169,8 @@ where id < 5 limit 5
    partitions=4/4 files=4 size=6.32KB
    runtime filters: RF000 -> a.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:SELECT
 |  predicates: a.id < 5
 |  limit: 5
@@ -179,6 +199,8 @@ where id < 5
 order by id
 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=5]
 |  order by: id ASC
 |
@@ -189,6 +211,8 @@ limit 5
    partitions=4/4 files=4 size=6.32KB
    limit: 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=5]
 |  order by: id ASC
 |
@@ -213,6 +237,8 @@ where id < 5
 order by id
 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=5]
 |  order by: id ASC
 |
@@ -231,6 +257,8 @@ limit 5
    partitions=4/4 files=4 size=6.32KB
    runtime filters: RF000 -> a.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=5]
 |  order by: id ASC
 |
@@ -259,6 +287,8 @@ select *
 from functional.alltypes
   join (select id from functional.alltypessmall limit 10) a using (id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypes.id = id
 |  runtime filters: RF000 <- id
@@ -271,6 +301,8 @@ from functional.alltypes
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -296,6 +328,8 @@ select *
 from functional.alltypes
   join (select id from functional.alltypessmall order by id limit 10) a using (id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypes.id = id
 |  runtime filters: RF000 <- id
@@ -310,6 +344,8 @@ from functional.alltypes
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |
 03:HASH JOIN [INNER JOIN, BROADCAST]
@@ -344,6 +380,8 @@ where a.id < 5
 order by a.id
 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:TOP-N [LIMIT=5]
 |  order by: id ASC
 |
@@ -368,6 +406,8 @@ limit 5
    predicates: functional.alltypes.id < 5
    runtime filters: RF000 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: id ASC
 |  limit: 5
@@ -416,6 +456,8 @@ where a.id < 5
 order by a.id
 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=5]
 |  order by: id ASC
 |
@@ -442,6 +484,8 @@ limit 5
    predicates: functional.alltypes.id < 5
    runtime filters: RF000 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: id ASC
 |  limit: 5
@@ -486,6 +530,8 @@ select x.id from (
 order by x.id
 limit 100 offset 4
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=100 OFFSET=4]
 |  order by: id ASC
 |
@@ -495,6 +541,8 @@ limit 100 offset 4
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=100 OFFSET=4]
 |  order by: id ASC
 |
@@ -519,6 +567,8 @@ left outer join
 on (a.id = b.id)
 where a.id > 10 and b.id > 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: id = id
 |  other predicates: id > 20
@@ -542,6 +592,8 @@ left outer join
 on (a.id = b.id)
 where a.id > 10 and b.id > 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: id = id
 |  other predicates: id > 20
@@ -570,6 +622,8 @@ right outer join
 on (a.id = b.id)
 where a.id > 10 and b.id > 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: id = id
 |  other predicates: id > 10
@@ -595,6 +649,8 @@ right outer join
 on (a.id = b.id)
 where a.id > 10 and b.id > 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: id = id
 |  other predicates: id > 10
@@ -618,6 +674,8 @@ where a.id > 10 and b.id > 20
 # have explain_level=1
 select * from (select * from functional.alltypes limit 100) v where id < 10 limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SELECT
 |  predicates: functional.alltypes.id < 10
 |  limit: 1

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test b/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
index fe6ade8..13f6326 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/inline-view.test
@@ -12,6 +12,8 @@ from (
 join functional.alltypes t2 on (t1.int_col = t2.int_col)
 where month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t2.int_col = int_col
 |  runtime filters: RF000 <- int_col
@@ -31,6 +33,8 @@ where month = 1
    partitions=2/24 files=2 size=40.32KB
    runtime filters: RF000 -> t2.int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -67,8 +71,12 @@ where month = 1
 # simple full scan subquery
 select * from (select y x from (select id y from functional_hbase.alltypessmall) a) b
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
@@ -78,6 +86,8 @@ select * from (select t2.*
 from functional.testtbl t1 join functional.testtbl t2 using(id)
 where t1.zip = 94611) x
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t2.id
 |  runtime filters: RF000 <- t2.id
@@ -90,6 +100,8 @@ where t1.zip = 94611) x
    predicates: t1.zip = 94611
    runtime filters: RF000 -> t1.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -121,6 +133,8 @@ from
      and b.string_col = '15'
      and a.tinyint_col + b.tinyint_col < 15) x
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: a.id = b.id, a.int_col = b.int_col
 |  other predicates: a.tinyint_col = 15, a.day >= 6, a.tinyint_col + b.tinyint_col < 15
@@ -145,6 +159,8 @@ NODE 1:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=3/090301.txt 0:1620
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=4/090401.txt 0:1621
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -168,6 +184,8 @@ NODE 1:
 # predicate pushdown
 select * from (select * from functional_hbase.alltypessmall) a where id < 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HBASE [functional_hbase.alltypessmall]
    predicates: functional_hbase.alltypessmall.id < 5
 ====
@@ -188,6 +206,8 @@ and b.string_col = '15'
 and a.tinyint_col + b.tinyint_col < 15
 and b.id + 15 = 27
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: id = id, int_col = int_col
 |  other predicates: tinyint_col = 15, day >= 6, tinyint_col + tinyint_col < 15
@@ -202,6 +222,8 @@ and b.id + 15 = 27
    predicates: functional.alltypesagg.tinyint_col = 15, functional.alltypesagg.id + 15 = 27
    runtime filters: RF000 -> id, RF001 -> int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -240,6 +262,8 @@ and a.tinyint_col = 15
 and b.string_col = '15'
 and a.tinyint_col + b.tinyint_col < 15
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: id = id, int_col = int_col
 |  other predicates: tinyint_col = 15, tinyint_col + tinyint_col < 15
@@ -264,6 +288,8 @@ NODE 1:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=3/090301.txt 0:1620
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=4/090401.txt 0:1621
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -301,6 +327,8 @@ and x.float_col > 4.5
 and c.string_col < '7'
 and x.int_col + x.float_col + cast(c.string_col as float) < 1000
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: c.id = a.tinyint_col
 |  other predicates: a.int_col + b.float_col + CAST(c.string_col AS FLOAT) < 1000
@@ -337,6 +365,8 @@ NODE 2:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=3/090301.txt 0:1620
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypessmall/year=2009/month=4/090401.txt 0:1621
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -374,6 +404,8 @@ avg(tinyint_col)
 from (select * from functional.alltypesagg) a
 group by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*), min(functional.alltypesagg.tinyint_col), max(functional.alltypesagg.tinyint_col), sum(functional.alltypesagg.tinyint_col), avg(functional.alltypesagg.tinyint_col)
 |  group by: functional.alltypesagg.tinyint_col
@@ -394,6 +426,8 @@ NODE 0:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypesagg/year=2010/month=1/day=9/100109.txt 0:76263
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypesagg/year=2010/month=1/day=__HIVE_DEFAULT_PARTITION__/000000_0 0:72759
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -417,6 +451,8 @@ from functional.alltypesagg
 group by 1
 ) a
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*), min(tinyint_col), max(tinyint_col), sum(tinyint_col), avg(tinyint_col)
 |  group by: tinyint_col
@@ -424,6 +460,8 @@ group by 1
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -450,6 +488,8 @@ from
     limit 5
 ) y
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:TOP-N [LIMIT=5]
 |  order by: c2 ASC, c3 DESC
 |
@@ -459,6 +499,8 @@ from
 |
 00:SCAN HBASE [functional_hbase.alltypessmall]
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c2 ASC, c3 DESC
 |  limit: 5
@@ -496,14 +538,20 @@ from (
 order by 2,1 desc
 limit 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:EMPTYSET
 ====
 # distinct *
 select distinct *
 from (select distinct * from functional.testtbl) x
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  group by: functional.testtbl.id, functional.testtbl.name, functional.testtbl.zip
 |
@@ -513,6 +561,8 @@ from (select distinct * from functional.testtbl) x
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:AGGREGATE [FINALIZE]
@@ -533,6 +583,8 @@ from (select distinct * from functional.testtbl) x
 select distinct id, zip
 from (select distinct * from functional.testtbl) x
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  group by: functional.testtbl.id, functional.testtbl.zip
 |
@@ -542,6 +594,8 @@ from (select distinct * from functional.testtbl) x
 00:SCAN HDFS [functional.testtbl]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 06:AGGREGATE [FINALIZE]
@@ -576,6 +630,8 @@ from (
 where c1 is not null
 and   c2 > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*), avg(functional.alltypesagg.int_col)
 |  group by: functional.alltypesagg.int_col % 7
@@ -584,6 +640,8 @@ and   c2 > 10
 00:SCAN HDFS [functional.alltypesagg]
    partitions=11/11 files=11 size=814.73KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 03:AGGREGATE [FINALIZE]
@@ -614,6 +672,8 @@ from (
      on (j.test_name = d.name)
 where j.test_id <= 1006
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: b.name = a.test_name
 |  runtime filters: RF000 <- a.test_name
@@ -626,6 +686,8 @@ where j.test_id <= 1006
    partitions=1/1 files=1 size=171B
    runtime filters: RF000 -> b.name
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -663,6 +725,8 @@ from functional.alltypessmall c
      on (x.tinyint_col = c.id)
 group by x.smallint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(b.id)
 |  group by: a.smallint_col
@@ -685,6 +749,8 @@ group by x.smallint_col
    partitions=11/11 files=11 size=814.73KB
    runtime filters: RF000 -> a.tinyint_col, RF001 -> a.smallint_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:EXCHANGE [UNPARTITIONED]
 |
 10:AGGREGATE [FINALIZE]
@@ -746,6 +812,8 @@ and x.float_col > 4.5
 and c.string_col < '7'
 and x.int_col + x.float_col + CAST(c.string_col AS FLOAT) < 1000
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: c.id = a.tinyint_col
 |  other predicates: a.int_col + b.float_col + CAST(c.string_col AS FLOAT) < 1000
@@ -769,6 +837,8 @@ and x.int_col + x.float_col + CAST(c.string_col AS FLOAT) < 1000
    predicates: c.string_col < '7'
    runtime filters: RF000 -> c.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -812,6 +882,8 @@ from functional.alltypessmall c
   ) x on (x.smallint_col = c.id)
 group by x.smallint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum(count(a.id))
 |  group by: b.smallint_col
@@ -839,6 +911,8 @@ group by x.smallint_col
    partitions=4/4 files=4 size=6.32KB
    runtime filters: RF000 -> c.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 13:EXCHANGE [UNPARTITIONED]
 |
 12:AGGREGATE [FINALIZE]
@@ -887,9 +961,13 @@ group by x.smallint_col
 # Values statement in subqueries with predicate
 select * from (select y from (values((1 as y),(11))) a where y < 10) b
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=1
 ====
@@ -902,6 +980,8 @@ select * from
      (select tinyint_col from functional.alltypes)) a
    where y < 10) b
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  constant-operands=1
 |
@@ -909,6 +989,8 @@ select * from
    partitions=24/24 files=24 size=478.45KB
    predicates: functional.alltypes.tinyint_col < 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 02:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -922,9 +1004,13 @@ select * from
 select * from (select 1 as y union all select 2 union all select * from (select 11) a) b
 where y < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=2
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=2
 ====
@@ -933,11 +1019,15 @@ where y < 10
 select * from (values(1 as y) union all values(2) union all select * from (values(11)) a) b
 where y < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  constant-operands=2
 |
 01:UNION
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  constant-operands=2
 |
@@ -951,6 +1041,8 @@ inner join
 inner join
 (select 1 a, 3 b union all select 1 a, 3 b) z on z.b = y.b
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: b = b
 |
@@ -966,6 +1058,8 @@ inner join
 00:UNION
    constant-operands=2
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN, BROADCAST]
 |  hash predicates: b = b
 |
@@ -992,6 +1086,8 @@ left semi join
 inner join
 (select 1 a, 3 b union all select 1 a, 3 b) z on z.b = x.id + 2
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: x.id + 2 = b
 |  runtime filters: RF000 <- b
@@ -1010,6 +1106,8 @@ inner join
    partitions=4/4 files=4 size=6.32KB
    runtime filters: RF000 -> x.id + 2, RF001 -> x.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1039,6 +1137,8 @@ select b.* from functional.decimal_tbl a left outer join
   (select d1, d1 + NULL IS NULL x from functional.decimal_tbl) b
 on (a.d1 = b.d1)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.d1 = d1
 |
@@ -1056,6 +1156,8 @@ where foo = 10
 group by foo
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:AGGREGATE [FINALIZE]
 |  output: sum(foo)
 |  group by: foo
@@ -1075,6 +1177,8 @@ select * from
    from functional.alltypestiny t1 left outer join functional.alltypes t2
    on t1.int_col = t2.int_col and t1.tinyint_col = t2.tinyint_col) t
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: t2.int_col = t1.int_col, t2.tinyint_col = t1.tinyint_col
 |  runtime filters: RF000 <- t1.int_col, RF001 <- t1.tinyint_col
@@ -1096,6 +1200,8 @@ select 1 from
 inner join functional.alltypestiny c
 on (aid < bid and aid = c.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = c.id
 |  runtime filters: RF000 <- c.id
@@ -1131,6 +1237,8 @@ inner join
    on a.id = b.int_col) v
 on (t1.id = v.id and v.int_col is null and v.int_col < 10 and v.id < 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = t1.id
 |  runtime filters: RF000 <- t1.id
@@ -1161,6 +1269,8 @@ select * from
      from functional.alltypestiny) iv
   ) ivv
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
 ====
@@ -1171,6 +1281,8 @@ select * from
      from functional.alltypestiny) iv
   ) ivv where bigint_col = bigint_col2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
    predicates: bigint_col = bigint_col
@@ -1183,6 +1295,8 @@ select * from
      from functional.alltypestiny) iv
   ) ivv
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: sum(bigint_col)
 |
@@ -1196,6 +1310,8 @@ select * from
      from functional.alltypestiny) iv
   ) ivv where s1 = s2
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: sum(bigint_col)
 |  having: sum(bigint_col) = sum(bigint_col)

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test b/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
index e1951e0..49a0b87 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/join-order.test
@@ -23,6 +23,8 @@ order by
   o_orderdate
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=10]
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 5) DESC, o_orderdate ASC
 |
@@ -52,6 +54,8 @@ limit 10
    predicates: l_shipdate > '1995-03-15'
    runtime filters: RF001 -> l.l_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 5) DESC, o_orderdate ASC
 |  limit: 10
@@ -120,6 +124,8 @@ order by
   o_orderdate
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=10]
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 5) DESC, o_orderdate ASC
 |
@@ -149,6 +155,8 @@ limit 10
    predicates: c.c_mktsegment = 'BUILDING'
    runtime filters: RF001 -> c.c_custkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 5) DESC, o_orderdate ASC
 |  limit: 10
@@ -219,6 +227,8 @@ order by
   revenue desc
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:TOP-N [LIMIT=100]
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 5) DESC
 |
@@ -271,6 +281,8 @@ limit 100
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF003 -> l_suppkey, RF005 -> l_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 5) DESC
 |  limit: 100
@@ -369,6 +381,8 @@ where
   and s.s_nationkey = n.n_nationkey
   and n.n_regionkey = r.r_regionkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [INNER JOIN]
 |  hash predicates: n.n_regionkey = r.r_regionkey
 |  runtime filters: RF000 <- r.r_regionkey
@@ -405,6 +419,8 @@ where
    partitions=1/1 files=1 size=1.33MB
    runtime filters: RF001 -> s.s_nationkey, RF002 -> s.s_suppkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 13:EXCHANGE [UNPARTITIONED]
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
@@ -469,6 +485,8 @@ order by
   o_orderpriority
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=10]
 |  order by: o_orderpriority ASC
 |
@@ -489,6 +507,8 @@ limit 10
    predicates: l_commitdate < l_receiptdate
    runtime filters: RF000 -> l_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_orderpriority ASC
 |  limit: 10
@@ -531,6 +551,8 @@ from tpch.orders
 group by o_orderpriority
 order by o_orderpriority limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=10]
 |  order by: o_orderpriority ASC
 |
@@ -547,6 +569,8 @@ order by o_orderpriority limit 10
 00:SCAN HDFS [tpch.orders]
    partitions=1/1 files=1 size=162.56MB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_orderpriority ASC
 |  limit: 10
@@ -585,6 +609,8 @@ from tpch.orders
 group by o_orderpriority
 order by o_orderpriority limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=10]
 |  order by: o_orderpriority ASC
 |
@@ -603,6 +629,8 @@ order by o_orderpriority limit 10
    partitions=1/1 files=1 size=162.56MB
    runtime filters: RF000 -> o_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_orderpriority ASC
 |  limit: 10
@@ -643,6 +671,8 @@ from tpch.customer
   join tpch.nation on (c_nationkey = n_nationkey)
 where n_name = 'x'
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -666,6 +696,8 @@ where n_name = 'x'
    partitions=1/1 files=1 size=162.56MB
    runtime filters: RF001 -> o_custkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |
@@ -708,6 +740,8 @@ from tpch.customer
   join tpch.nation on (c_nationkey = n_nationkey)
 where n_name = 'x'
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -728,6 +762,8 @@ where n_name = 'x'
    partitions=1/1 files=1 size=23.08MB
    runtime filters: RF000 -> c_nationkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |
@@ -765,6 +801,8 @@ select c.int_col from functional.alltypestiny a
 cross join functional.alltypestiny b
 cross join functional.alltypes c
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--03:NESTED LOOP JOIN [CROSS JOIN]
@@ -788,6 +826,8 @@ right join functional.alltypesagg t4 on (t3.id = t4.id)
 inner join functional.alltypes t5 on (t4.id = t5.id)
 inner join functional.alltypestiny t6 on (t5.id = t6.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 11:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -843,6 +883,8 @@ left semi join functional.alltypesagg t4 on (t3.id = t4.id)
 inner join functional.alltypes t5 on (t3.id = t5.id)
 right join functional.alltypestiny t6 on (t5.id = t6.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 13:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -897,6 +939,8 @@ inner join functional.alltypessmall t4 on (t3.id = t4.id)
 left semi join functional.alltypes t5 on (t4.id = t5.id)
 inner join functional.alltypestiny t6 on (t3.id = t6.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 13:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -952,6 +996,8 @@ inner join functional.alltypessmall t4 on (t3.id = t4.id)
 left anti join functional.alltypes t5 on (t4.id = t5.id)
 inner join functional.alltypestiny t6 on (t3.id = t6.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 13:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -1012,6 +1058,8 @@ on (t2.id = t3.id)
 inner join functional.alltypestiny t4
 on (t3.id = t4.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:HASH JOIN [INNER JOIN]
 |  hash predicates: t3.id = t4.id
 |  runtime filters: RF000 <- t4.id
@@ -1075,6 +1123,8 @@ on (t2.id = t3.id)
 inner join functional.alltypestiny t4
 on (t3.id = t4.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:HASH JOIN [INNER JOIN]
 |  hash predicates: t3.id = t4.id
 |  runtime filters: RF000 <- t4.id
@@ -1137,6 +1187,8 @@ inner join functional.alltypestiny t4
 on (t2.id = t4.id)
 where t2.month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:HASH JOIN [INNER JOIN]
 |  hash predicates: b.id = t4.id
 |  runtime filters: RF000 <- t4.id
@@ -1201,6 +1253,8 @@ inner join functional.alltypestiny t4
 on (t2.id = t4.id)
 where t2.month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:HASH JOIN [INNER JOIN]
 |  hash predicates: b.id = t4.id
 |  runtime filters: RF000 <- t4.id
@@ -1264,6 +1318,8 @@ LEFT SEMI JOIN
 ON t4.bigint_col = `$a$3`.`$c$1`
 WHERE `$a$2`.`$c$1` > t4.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:AGGREGATE [FINALIZE]
 |  output: sum(t4.tinyint_col)
 |
@@ -1310,6 +1366,8 @@ left outer join functional.alltypessmall b
 # both predicates should appear in the 'other predicates'
 where a.int_col = b.int_col and b.bigint_col < a.tinyint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: b.id = a.id
 |  other predicates: a.int_col = b.int_col, b.bigint_col < a.tinyint_col
@@ -1333,6 +1391,8 @@ right outer join functional.alltypes c
 where a.int_col = b.int_col and b.bigint_col < a.tinyint_col
   and b.tinyint_col = c.tinyint_col and b.bool_col != c.bool_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: c.id = b.id
 |  other predicates: a.int_col = b.int_col, b.bool_col != c.bool_col, b.tinyint_col = c.tinyint_col, b.bigint_col < a.tinyint_col
@@ -1363,6 +1423,8 @@ right outer join functional.alltypes c
 # all predicates should appear in the 'other predicates'
 where b.tinyint_col = c.tinyint_col and b.bool_col != c.bool_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: c.id = b.id
 |  other predicates: b.bool_col != c.bool_col, b.tinyint_col = c.tinyint_col
@@ -1393,6 +1455,8 @@ select count(1) from
 left outer join functional.alltypestiny t3
 on (t3.string_col = t1.string_col_1 and t3.date_string_col = t1.string_col_2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  output: count(1)
 |
@@ -1427,6 +1491,8 @@ where timestamp_col = now()) b
 on (a.id = b.id)
 and a.date_string_col = ''
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: a.id = functional.alltypes.id
 |  runtime filters: RF000 <- functional.alltypes.id
@@ -1447,6 +1513,8 @@ where date_string_col = '') b
 on (a.id = b.id)
 and a.timestamp_col = now()
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: functional.alltypes.id = a.id
 |  runtime filters: RF000 <- a.id
@@ -1468,6 +1536,8 @@ where timestamp_col <=> now()) b
 on (a.id = b.id)
 and a.date_string_col <=> ''
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: a.id = functional.alltypes.id
 |  runtime filters: RF000 <- functional.alltypes.id
@@ -1488,6 +1558,8 @@ where date_string_col <=> '') b
 on (a.id = b.id)
 and a.timestamp_col <=> now()
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: functional.alltypes.id = a.id
 |  runtime filters: RF000 <- a.id

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
index 260ba21..ba1395a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/joins.test
@@ -2,6 +2,8 @@ select *
 from functional.testtbl t1 join functional.testtbl t2 using(id)
 where t1.zip = 94611
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t2.id
 |  runtime filters: RF000 <- t2.id
@@ -14,6 +16,8 @@ where t1.zip = 94611
    predicates: t1.zip = 94611
    runtime filters: RF000 -> t1.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -36,6 +40,8 @@ from functional.testtbl t1 left outer join functional.testtbl t2
 on (t1.id - 1 = t2.id + 1)
 where t1.zip = 94611
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: t1.id - 1 = t2.id + 1
 |
@@ -46,6 +52,8 @@ where t1.zip = 94611
    partitions=1/1 files=0 size=0B
    predicates: t1.zip = 94611
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [LEFT OUTER JOIN, BROADCAST]
@@ -67,6 +75,8 @@ from (select * from functional.alltypestiny) t1
   join (select * from functional.alltypestiny) t2 on (t1.id = t2.id)
   join functional.alltypestiny t3 on (coalesce(t1.id, t2.id) = t3.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: coalesce(functional.alltypestiny.id, functional.alltypestiny.id) = t3.id
 |  runtime filters: RF000 <- t3.id
@@ -86,6 +96,8 @@ from (select * from functional.alltypestiny) t1
    partitions=4/4 files=4 size=460B
    runtime filters: RF000 -> coalesce(functional.alltypestiny.id, functional.alltypestiny.id), RF001 -> functional.alltypestiny.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -122,6 +134,8 @@ and a.tinyint_col = 15
 and b.string_col = '15'
 and a.tinyint_col + b.tinyint_col < 15
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: a.id = b.id, a.int_col = b.int_col
 |  other predicates: a.tinyint_col = 15, a.day >= 6, a.tinyint_col + b.tinyint_col < 15
@@ -136,6 +150,8 @@ and a.tinyint_col + b.tinyint_col < 15
    predicates: a.tinyint_col = 15
    runtime filters: RF000 -> a.id, RF001 -> a.int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -171,6 +187,8 @@ and a.tinyint_col + b.tinyint_col < 15
 and a.float_col - c.double_col < 0
 and (b.double_col * c.tinyint_col > 1000 or c.tinyint_col < 1000)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: c.id = a.id, c.string_col = b.string_col
 |  other predicates: a.tinyint_col = 15, b.string_col = '15', a.day >= 6, b.month > 2, a.float_col - c.double_col < 0, a.tinyint_col + b.tinyint_col < 15, (b.double_col * c.tinyint_col > 1000 OR c.tinyint_col < 1000)
@@ -189,6 +207,8 @@ and (b.double_col * c.tinyint_col > 1000 or c.tinyint_col < 1000)
 02:SCAN HDFS [functional.alltypesaggnonulls c]
    partitions=2/10 files=2 size=148.10KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [LEFT OUTER JOIN, PARTITIONED]
@@ -224,6 +244,8 @@ inner join
 (select 1 as x, id from functional.alltypessmall) b
 on a.x = b.x
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: 1 = 1
 |
@@ -237,6 +259,8 @@ on a.x = b.x
 select a.int_col, b.x from functional.alltypessmall a inner join
 (values(1 as int_col, 'a' as x), (1, 'b'), (2, 'c')) b on a.int_col = b.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: a.int_col = int_col
 |  runtime filters: RF000 <- int_col
@@ -248,6 +272,8 @@ select a.int_col, b.x from functional.alltypessmall a inner join
    partitions=4/4 files=4 size=6.32KB
    runtime filters: RF000 -> a.int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -267,6 +293,8 @@ select a.int_col, b.x from functional.alltypessmall a inner join
 select *
 from functional.alltypesagg join functional_hbase.alltypessmall using (id, int_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypesagg.id = functional_hbase.alltypessmall.id, functional.alltypesagg.int_col = functional_hbase.alltypessmall.int_col
 |  runtime filters: RF000 <- functional_hbase.alltypessmall.id, RF001 <- functional_hbase.alltypessmall.int_col
@@ -277,6 +305,8 @@ from functional.alltypesagg join functional_hbase.alltypessmall using (id, int_c
    partitions=11/11 files=11 size=814.73KB
    runtime filters: RF000 -> functional.alltypesagg.id, RF001 -> functional.alltypesagg.int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -302,6 +332,8 @@ and b.tinyint_col = 5
 and b.tinyint_col > 123
 and a.tinyint_col + b.tinyint_col < 15
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: a.int_col = b.int_col, a.id = CAST(b.id AS INT)
 |  other predicates: a.tinyint_col + b.tinyint_col < 15
@@ -326,6 +358,8 @@ NODE 0:
 NODE 1:
   HBASE KEYRANGE port=16202 5:5\0
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -361,6 +395,8 @@ group by x.tinyint_col
 order by 2
 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=5]
 |  order by: count(x.day) ASC
 |
@@ -401,6 +437,8 @@ limit 5
    partitions=11/11 files=11 size=814.73KB
    runtime filters: RF000 -> d.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=5]
 |  order by: count(x.day) ASC
 |
@@ -456,6 +494,8 @@ limit 5
 # join without "other join conjuncts"
 select * from functional.alltypessmall a, functional.alltypessmall b where a.id = b.id limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = b.id
 |  runtime filters: RF000 <- b.id
@@ -468,6 +508,8 @@ select * from functional.alltypessmall a, functional.alltypessmall b where a.id
    partitions=4/4 files=4 size=6.32KB
    runtime filters: RF000 -> a.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:EXCHANGE [UNPARTITIONED]
 |  limit: 1
 |
@@ -493,6 +535,8 @@ select *
 from functional.testtbl t1, functional.testtbl t2, functional.testtbl t3
 where t1.id = t3.id and t2.id = t3.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id = t3.id
 |  runtime filters: RF000 <- t3.id
@@ -512,6 +556,8 @@ where t1.id = t3.id and t2.id = t3.id
    partitions=1/1 files=0 size=0B
    runtime filters: RF000 -> t1.id, RF001 -> t1.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -542,6 +588,8 @@ where t1.id = t3.id and t2.id = t3.id
 select * from functional.emptytable a inner join
 functional.alltypes b on a.f2 = b.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: b.int_col = a.f2
 |  runtime filters: RF000 <- a.f2
@@ -553,6 +601,8 @@ functional.alltypes b on a.f2 = b.int_col
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> b.int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]
@@ -572,6 +622,8 @@ functional.alltypes b on a.f2 = b.int_col
 select *
 from functional.testtbl t1 cross join functional.testtbl
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--01:SCAN HDFS [functional.testtbl]
@@ -580,6 +632,8 @@ from functional.testtbl t1 cross join functional.testtbl
 00:SCAN HDFS [functional.testtbl t1]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
@@ -596,6 +650,8 @@ from functional.testtbl t1 cross join functional.testtbl
 select *
 from functional.testtbl t1 cross join functional.testtbl t2 where t1.id < t2.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: t1.id < t2.id
 |
@@ -605,6 +661,8 @@ from functional.testtbl t1 cross join functional.testtbl t2 where t1.id < t2.id
 00:SCAN HDFS [functional.testtbl t1]
    partitions=1/1 files=0 size=0B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -626,6 +684,8 @@ on (a.id = b.id and b.int_col = a.int_col)
 inner join [shuffle] functional.alltypes c
 on (b.id = c.id and c.int_col = b.int_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: b.id = c.id, b.int_col = c.int_col
 |  runtime filters: RF000 <- c.id, RF001 <- c.int_col
@@ -645,6 +705,8 @@ on (b.id = c.id and c.int_col = b.int_col)
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> a.id, RF001 -> a.int_col, RF002 -> a.id, RF003 -> a.int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -680,6 +742,8 @@ inner join [shuffle]
    from functional.alltypes group by int_col, bool_col) b
 on (a.int_col = b.int_col and b.bool_col = a.bool_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:HASH JOIN [INNER JOIN]
 |  hash predicates: a.bool_col = bool_col, a.int_col = int_col
 |  runtime filters: RF000 <- bool_col, RF001 <- int_col
@@ -695,6 +759,8 @@ on (a.int_col = b.int_col and b.bool_col = a.bool_col)
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> a.bool_col, RF001 -> a.int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 03:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -730,6 +796,8 @@ inner join [shuffle]
    from functional.alltypes group by int_col, bool_col) c
 on (b.int_col = c.int_col and c.bool_col = b.bool_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [INNER JOIN]
 |  hash predicates: b.bool_col = a.bool_col, b.int_col = a.int_col
 |  runtime filters: RF000 <- a.bool_col, RF001 <- a.int_col
@@ -753,6 +821,8 @@ on (b.int_col = c.int_col and c.bool_col = b.bool_col)
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> b.bool_col, RF001 -> b.int_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:EXCHANGE [UNPARTITIONED]
 |
 05:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -801,6 +871,8 @@ and b.string_col = a.string_col and b.date_string_col = a.string_col
 # redundant predicates to test minimal spanning tree of equivalent slots at a
 where a.tinyint_col = a.smallint_col and a.int_col = a.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: b.id = a.id, b.string_col = a.string_col
 |  runtime filters: RF000 <- a.id, RF001 <- a.string_col
@@ -826,6 +898,8 @@ and b.string_col = a.string_col and b.date_string_col = a.string_col
 # redundant predicates to test minimal spanning tree of equivalent slots at a
 where a.tinyint_col = a.smallint_col and a.int_col = a.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: b.id = a.id, b.int_col = a.id, b.id = a.int_col, b.id = a.bigint_col, b.bigint_col = a.id, b.id = a.smallint_col, b.string_col = a.string_col, b.id = a.tinyint_col, b.date_string_col = a.string_col
 |  runtime filters: RF000 <- a.id, RF001 <- a.id, RF002 <- a.int_col, RF003 <- a.bigint_col, RF004 <- a.id, RF005 <- a.smallint_col, RF006 <- a.string_col, RF007 <- a.tinyint_col, RF008 <- a.string_col
@@ -848,6 +922,8 @@ inner join functional.alltypessmall c
 # redundant join predicates
 where a.id = c.id and a.int_col = c.int_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: c.id = b.id, c.int_col = b.int_col
 |  runtime filters: RF000 <- b.id, RF001 <- b.int_col
@@ -877,6 +953,8 @@ functional.alltypes b,
 functional.alltypessmall c
 where a.id = c.id and b.int_col = c.int_col and b.int_col = c.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: c.id = a.id
 |  runtime filters: RF000 <- a.id
@@ -907,6 +985,8 @@ select a.* from
    group by 1, 2) b
 on a.int_col = b.int_col and a.int_col = b.smallint_col and a.int_col = b.c
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: int_col = int_col
 |  runtime filters: RF000 <- int_col
@@ -946,6 +1026,8 @@ and t3.int_col = t2.bigint_col
 # already been established by 't3.int_col = t2.bigint_col'
 and t3.bigint_col = t2.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.smallint_col = t3.int_col, t2.bigint_col = t3.int_col
 |  runtime filters: RF000 <- t3.int_col, RF001 <- t3.int_col
@@ -966,6 +1048,8 @@ and t3.bigint_col = t2.bigint_col
    partitions=11/11 files=11 size=814.73KB
    runtime filters: RF000 -> t2.smallint_col, RF001 -> t2.bigint_col, RF002 -> t2.smallint_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1006,6 +1090,8 @@ inner join
 on a.id = b.x and a.id = b.tinyint_col and
    a.int_col = b.y and a.int_col = b.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = id + id, a.int_col = int_col * int_col
 |  runtime filters: RF000 <- id + id, RF001 <- int_col * int_col
@@ -1031,6 +1117,8 @@ inner join
 on a.id = b.x and a.id = b.tinyint_col and
    a.int_col = b.y and a.int_col = b.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = id + id, a.int_col = int_col * int_col
 |  runtime filters: RF000 <- id + id, RF001 <- int_col * int_col
@@ -1061,6 +1149,8 @@ inner join
 on a.id = b.x and a.id = b.tinyint_col and
    a.int_col = b.y and a.int_col = b.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = id + id, a.int_col = int_col * int_col
 |  runtime filters: RF000 <- id + id, RF001 <- int_col * int_col
@@ -1098,6 +1188,8 @@ inner join
 on a.id = b.x and a.id = b.tinyint_col and
    a.int_col = b.y and a.int_col = b.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: a.id = tinyint_col, a.id = x, a.int_col = bigint_col, a.int_col = y
 |  runtime filters: RF000 <- tinyint_col, RF001 <- x, RF002 <- bigint_col, RF003 <- y
@@ -1123,6 +1215,8 @@ on t3.smallint_col = t1.tinyint_col
 inner join functional.alltypes t2
 on t2.string_col = t1.string_col and t3.date_string_col = t2.string_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.string_col = t2.string_col
 |  runtime filters: RF000 <- t2.string_col
@@ -1149,6 +1243,8 @@ full outer join
 (select distinct bigint_col from functional.alltypestiny) b
 on (a.int_col = b.bigint_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: int_col = bigint_col
 |
@@ -1164,6 +1260,8 @@ on (a.int_col = b.bigint_col)
 00:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
@@ -1206,6 +1304,8 @@ functional.alltypestiny b
 on a.id = b.id
 where b.id < 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -1246,6 +1346,8 @@ left outer join
 on a.id = b.id
 where b.id < 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [RIGHT OUTER JOIN, PARTITIONED]
@@ -1294,6 +1396,8 @@ full outer join
 on a.id = b.id
 where b.id < 5
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 06:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
@@ -1329,6 +1433,8 @@ where b.id < 5
 select straight_join count(*)
 from functional.decimal_tbl a join functional.decimal_tbl b on a.d1 = b.d5
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -1348,6 +1454,8 @@ select j.* from functional.jointbl j left semi join functional.dimtbl d
   on (j.test_id = d.id and j.test_zip < d.zip and d.name = 'Name2')
 where j.test_id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: j.test_id = d.id
 |  other join predicates: j.test_zip < d.zip
@@ -1367,6 +1475,8 @@ select b.* from functional.alltypes a right semi join functional.alltypestiny b
   on (a.id = b.id and a.int_col < b.int_col and a.bool_col = false)
 where b.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: a.id = b.id
 |  other join predicates: a.int_col < b.int_col
@@ -1386,6 +1496,8 @@ select j.* from functional.jointbl j left anti join functional.dimtbl d
   on (j.test_id = d.id and j.test_zip < d.zip and d.name = 'Name2')
 where j.test_id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT ANTI JOIN]
 |  hash predicates: j.test_id = d.id
 |  other join predicates: j.test_zip < d.zip
@@ -1403,6 +1515,8 @@ select count(*) from functional.JoinTbl j
 left anti join functional.DimTbl d on j.test_id = d.id
 inner join functional.JoinTbl k on j.test_id = k.test_id and j.alltypes_id = 5000
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -1429,6 +1543,8 @@ select straight_join * from functional.alltypestiny a
 inner join [broadcast] functional.alltypes b on a.id = b.id
 inner join [shuffle] functional.alltypessmall c on b.id = c.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1461,6 +1577,8 @@ select /* +straight_join */ * from functional.alltypestiny a
 inner join /* +broadcast */ functional.alltypes b on a.id = b.id
 inner join /* +shuffle */ functional.alltypessmall c on b.id = c.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1499,6 +1617,8 @@ inner join
 -- +shuffle
 functional.alltypessmall c on b.id = c.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1536,6 +1656,8 @@ cross join functional.alltypes c
 # assigned to inverted cross join
 where c.id != b.id
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: c.id != b.id
 |
@@ -1570,6 +1692,8 @@ on (v1.tinyint_col = v2.tinyint_col and
     v1.tinyint_col = v2.int_col and
     v1.tinyint_col = v2.bigint_col)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1620,6 +1744,8 @@ on (v1.tinyint_col = v2.tinyint_col and
     v1.tinyint_col = v2.int_col and
     v1.tinyint_col = v2.bigint_col)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
@@ -1666,6 +1792,8 @@ inner join
 on (v1.string_col = v2.string_col and
     v1.bigint_col = v2.bigint_col)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1706,6 +1834,8 @@ left semi join functional.alltypes b
 # predicates are in reverse order of compatible group by exprs
 on (a.string_col = b.string_col and a.int_col = b.int_col)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 03:HASH JOIN [LEFT SEMI JOIN, PARTITIONED]
@@ -1738,6 +1868,8 @@ right semi join
 # predicates are in reverse order of compatible group by exprs
 on (a.string_col = b.string_col and a.int_col = b.int_col)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 03:HASH JOIN [RIGHT SEMI JOIN, PARTITIONED]
@@ -1777,6 +1909,8 @@ inner join
     where bigint_col = smallint_col and smallint_col = tinyint_col) c
 on (b.int_col = c.smallint_col and b.string_col = c.string_col)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 16:EXCHANGE [UNPARTITIONED]
 |
 07:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -1842,6 +1976,8 @@ where not exists (select *
 # Predicate on c (invisible side of anti join) is assigned to the scan node
                   and c.int_col < 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT ANTI JOIN]
 |  hash predicates: a.id = c.id
 |  other join predicates: a.tinyint_col = 10, a.int_col = b.int_col
@@ -1875,6 +2011,8 @@ inner join functional.alltypesagg d
   on (a.tinyint_col = d.tinyint_col and a.int_col < 10)
 where a.float_col < b.float_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [INNER JOIN]
 |  hash predicates: d.tinyint_col = a.tinyint_col
 |  runtime filters: RF000 <- a.tinyint_col
@@ -1914,6 +2052,8 @@ inner join functional.alltypesagg d
   on b.tinyint_col > d.int_col or b.id != d.id
 where a.int_col = 10 and c.bigint_col = d.bigint_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: a.id < b.id, b.tinyint_col > d.int_col OR b.id != d.id
 |
@@ -1938,6 +2078,8 @@ where a.int_col = 10 and c.bigint_col = d.bigint_col
    partitions=11/11 files=11 size=814.73KB
    runtime filters: RF000 -> d.bigint_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:EXCHANGE [UNPARTITIONED]
 |
 06:NESTED LOOP JOIN [INNER JOIN, BROADCAST]
@@ -1985,6 +2127,8 @@ inner join
 on vv1.bigint_col > vv2.bigint_col
 where vv1.tinyint_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: a.bigint_col > c.bigint_col
 |
@@ -2021,6 +2165,8 @@ left anti join functional.alltypesagg e
   on c.string_col != e.string_col
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:NESTED LOOP JOIN [RIGHT ANTI JOIN]
 |  join predicates: c.string_col != e.string_col
 |
@@ -2055,6 +2201,8 @@ left join (select coalesce(1, 10) as int_col
 from functional.alltypessmall) t2 on t1.id = t2.int_col
 where t2.int_col in (t2.int_col, 10);
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(id)
 |
@@ -2074,6 +2222,8 @@ select *
 from functional.testtbl t1 join functional.testtbl t2
 where t1.id <=> t2.id and t1.zip = 94611
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id IS NOT DISTINCT FROM t2.id
 |  runtime filters: RF000 <- t2.id
@@ -2090,6 +2240,8 @@ select *
 from functional.testtbl t1 join functional.testtbl t2
 where t1.id is not distinct from t2.id and t1.zip = 94611
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [INNER JOIN]
 |  hash predicates: t1.id IS NOT DISTINCT FROM t2.id
 |  runtime filters: RF000 <- t2.id
@@ -2106,6 +2258,8 @@ select *
 from functional.testtbl t1 join functional.testtbl t2
 where (t1.id IS DISTINCT FROM t2.id) and t1.zip = 94611
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: (t1.id IS DISTINCT FROM t2.id)
 |
@@ -2122,6 +2276,8 @@ from (select * from functional.alltypestiny) t1
   join (select * from functional.alltypestiny) t2 on (t1.id is not distinct from t2.id)
   join functional.alltypestiny t3 on (coalesce(t1.id, t2.id) is not distinct from t3.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: coalesce(functional.alltypestiny.id, functional.alltypestiny.id) IS NOT DISTINCT FROM t3.id
 |
@@ -2152,6 +2308,8 @@ and a.tinyint_col + b.tinyint_col < 15
 and a.float_col - c.double_col < 0
 and (b.double_col * c.tinyint_col > 1000 or c.tinyint_col < 1000)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: c.id = a.id, c.string_col IS NOT DISTINCT FROM b.string_col
 |  other predicates: a.tinyint_col = 15, b.string_col = '15', a.day >= 6, b.month > 2, a.float_col - c.double_col < 0, a.tinyint_col + b.tinyint_col < 15, (b.double_col * c.tinyint_col > 1000 OR c.tinyint_col < 1000)
@@ -2173,6 +2331,8 @@ and (b.double_col * c.tinyint_col > 1000 or c.tinyint_col < 1000)
 select t1.d, t2.d from functional.nulltable t1, functional.nulltable t2
 where not(t1.d IS DISTINCT FROM t2.d)
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: NOT (t1.d IS DISTINCT FROM t2.d)
 |
@@ -2187,6 +2347,8 @@ from functional.nulltable t1, functional.nulltable t2, functional.nulltable t3
 where t1.d IS DISTINCT FROM t2.d
 and t3.a != t2.g
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:NESTED LOOP JOIN [INNER JOIN]
 |  predicates: t3.a != t2.g
 |
@@ -2207,6 +2369,8 @@ and t3.a != t2.g
 # have explain_level=1
 select a.c_custkey as c_custkey from tpch.customer a, tpch.customer b limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:NESTED LOOP JOIN [CROSS JOIN]
 |  limit: 1
 |
@@ -2219,6 +2383,8 @@ select a.c_custkey as c_custkey from tpch.customer a, tpch.customer b limit 1
 select a.c_custkey as c_custkey from tpch.customer a left semi join tpch.customer b
 using (c_custkey) limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT SEMI JOIN]
 |  hash predicates: a.c_custkey = b.c_custkey
 |  runtime filters: RF000 <- b.c_custkey
@@ -2234,6 +2400,8 @@ using (c_custkey) limit 1
 select b.c_custkey as c_custkey from tpch.customer a right semi join tpch.customer b
 using (c_custkey) limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: a.c_custkey = b.c_custkey
 |  runtime filters: RF000 <- b.c_custkey
@@ -2249,6 +2417,8 @@ using (c_custkey) limit 1
 select a.c_custkey as c_custkey from tpch.customer a left outer join tpch.customer b
 using (c_custkey) limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT OUTER JOIN]
 |  hash predicates: a.c_custkey = b.c_custkey
 |  limit: 1
@@ -2262,6 +2432,8 @@ using (c_custkey) limit 1
 select b.c_custkey as c_custkey from tpch.customer a right outer join tpch.customer b
 using (c_custkey) limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: a.c_custkey = b.c_custkey
 |  runtime filters: RF000 <- b.c_custkey
@@ -2277,6 +2449,8 @@ using (c_custkey) limit 1
 select a.c_custkey as c_custkey from tpch.customer a full outer join tpch.customer b
 using (c_custkey) limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: a.c_custkey = b.c_custkey
 |  limit: 1
@@ -2290,6 +2464,8 @@ using (c_custkey) limit 1
 select a.c_custkey as c_custkey from tpch.customer a left anti join tpch.customer b
 using (c_custkey) limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [LEFT ANTI JOIN]
 |  hash predicates: a.c_custkey = b.c_custkey
 |  limit: 1
@@ -2303,6 +2479,8 @@ using (c_custkey) limit 1
 select b.c_custkey as c_custkey from tpch.customer a right anti join tpch.customer b
 using (c_custkey) limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:HASH JOIN [RIGHT ANTI JOIN]
 |  hash predicates: a.c_custkey = b.c_custkey
 |  limit: 1

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
index 73acf7f..a187ed8 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu-selectivity.test
@@ -1,12 +1,16 @@
 select * from functional_kudu.zipcode_incomes where id = '8600000US00601'
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED]
+  PLAN-ROOT SINK
+  |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      kudu predicates: id = '8600000US00601'
      hosts=3 per-host-mem=unavailable
      tuple-ids=0 row-size=124B cardinality=1
 ---- DISTRIBUTEDPLAN
 F01:PLAN FRAGMENT [UNPARTITIONED]
+  PLAN-ROOT SINK
+  |
   01:EXCHANGE [UNPARTITIONED]
      hosts=3 per-host-mem=unavailable
      tuple-ids=0 row-size=124B cardinality=1
@@ -22,6 +26,8 @@ F00:PLAN FRAGMENT [RANDOM]
 select * from functional_kudu.zipcode_incomes where id != '1' and zip = '2'
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED]
+  PLAN-ROOT SINK
+  |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      predicates: id != '1'
      kudu predicates: zip = '2'
@@ -29,6 +35,8 @@ F00:PLAN FRAGMENT [UNPARTITIONED]
      tuple-ids=0 row-size=124B cardinality=1
 ---- DISTRIBUTEDPLAN
 F01:PLAN FRAGMENT [UNPARTITIONED]
+  PLAN-ROOT SINK
+  |
   01:EXCHANGE [UNPARTITIONED]
      hosts=3 per-host-mem=unavailable
      tuple-ids=0 row-size=124B cardinality=1
@@ -44,12 +52,16 @@ F00:PLAN FRAGMENT [RANDOM]
 select * from functional_kudu.zipcode_incomes where id > '1' and zip > '2'
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED]
+  PLAN-ROOT SINK
+  |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      kudu predicates: zip > '2', id > '1'
      hosts=3 per-host-mem=unavailable
      tuple-ids=0 row-size=124B cardinality=3317
 ---- DISTRIBUTEDPLAN
 F01:PLAN FRAGMENT [UNPARTITIONED]
+  PLAN-ROOT SINK
+  |
   01:EXCHANGE [UNPARTITIONED]
      hosts=3 per-host-mem=unavailable
      tuple-ids=0 row-size=124B cardinality=3317
@@ -64,12 +76,16 @@ F00:PLAN FRAGMENT [RANDOM]
 select * from functional_kudu.zipcode_incomes where id = '1' or id = '2'
 ---- PLAN
 F00:PLAN FRAGMENT [UNPARTITIONED]
+  PLAN-ROOT SINK
+  |
   00:SCAN KUDU [functional_kudu.zipcode_incomes]
      predicates: id = '1' OR id = '2'
      hosts=3 per-host-mem=unavailable
      tuple-ids=0 row-size=124B cardinality=2
 ---- DISTRIBUTEDPLAN
 F01:PLAN FRAGMENT [UNPARTITIONED]
+  PLAN-ROOT SINK
+  |
   01:EXCHANGE [UNPARTITIONED]
      hosts=3 per-host-mem=unavailable
      tuple-ids=0 row-size=124B cardinality=2

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/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 565f3a3..06ce157 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/kudu.test
@@ -1,5 +1,7 @@
 select * from functional_kudu.testtbl
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN KUDU [functional_kudu.testtbl]
 ---- SCANRANGELOCATIONS
 NODE 0:
@@ -7,12 +9,16 @@ NODE 0:
   ScanToken{table=testtbl, range-partition: [(int64 id=1007), <end>)}
   ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN KUDU [functional_kudu.testtbl]
 ====
 select * from functional_kudu.testtbl where name = '10'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN KUDU [functional_kudu.testtbl]
    kudu predicates: name = '10'
 ---- SCANRANGELOCATIONS
@@ -21,6 +27,8 @@ NODE 0:
   ScanToken{table=testtbl, range-partition: [(int64 id=1007), <end>)}
   ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN KUDU [functional_kudu.testtbl]
@@ -28,6 +36,8 @@ NODE 0:
 ====
 select * from functional_kudu.testtbl where name = NULL
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN KUDU [functional_kudu.testtbl]
    predicates: name = NULL
 ====
@@ -95,12 +105,16 @@ select * from functional_kudu.testtbl
 where id >= 10 and zip <= 5 and 20 >= id and 'foo' = name and zip >= 0 and 30 >= zip
 and zip > 1 and zip < 50
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN KUDU [functional_kudu.testtbl]
    kudu predicates: id <= 20, zip <= 30, id >= 10, zip < 50, zip <= 5, zip > 1, zip >= 0, name = 'foo'
 ---- SCANRANGELOCATIONS
 NODE 0:
   ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN KUDU [functional_kudu.testtbl]
@@ -110,6 +124,8 @@ NODE 0:
 select * from functional_kudu.testtbl
 where id < 10 + 30  and cast(sin(id) as boolean) = true and 20 * 3 >= id and 10 * 10 + 3 > id
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN KUDU [functional_kudu.testtbl]
    predicates: CAST(sin(id) AS BOOLEAN) = TRUE
    kudu predicates: id <= 60, id < 40, id < 103
@@ -117,6 +133,8 @@ where id < 10 + 30  and cast(sin(id) as boolean) = true and 20 * 3 >= id and 10
 NODE 0:
   ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN KUDU [functional_kudu.testtbl]
@@ -127,6 +145,8 @@ NODE 0:
 select * from functional_kudu.testtbl
 where cast(sin(id) as boolean) = true and name = 'a'
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN KUDU [functional_kudu.testtbl]
    predicates: CAST(sin(id) AS BOOLEAN) = TRUE
    kudu predicates: name = 'a'
@@ -136,6 +156,8 @@ NODE 0:
   ScanToken{table=testtbl, range-partition: [(int64 id=1007), <end>)}
   ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN KUDU [functional_kudu.testtbl]
@@ -147,6 +169,8 @@ NODE 0:
 select * from functional_kudu.testtbl
 where cast(sin(id) as boolean) = true and name is null
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN KUDU [functional_kudu.testtbl]
    predicates: name IS NULL, CAST(sin(id) AS BOOLEAN) = TRUE
 ---- SCANRANGELOCATIONS
@@ -155,6 +179,8 @@ NODE 0:
   ScanToken{table=testtbl, range-partition: [(int64 id=1007), <end>)}
   ScanToken{table=testtbl, range-partition: [<start>, (int64 id=1003))}
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN KUDU [functional_kudu.testtbl]
@@ -163,6 +189,8 @@ NODE 0:
 # IMPALA-3856: KuduScanNode crash when pushing predicates including a cast
 select o_orderkey from tpch_kudu.orders where o_orderkey < 10.0 order by 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SORT
 |  order by: o_orderkey ASC
 |
@@ -174,6 +202,8 @@ select t.c from
   (select cast(o_orderdate as timestamp) c from tpch_kudu.orders where o_orderkey < 10) t
 where t.c <= cast('1995-01-01 00:00:00' as timestamp) order by c
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SORT
 |  order by: c ASC
 |
@@ -186,6 +216,8 @@ select count(*) from functional_kudu.alltypes
 where id < 1475059765 + 10
 and 1475059765 + 100 < id
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(*)
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test b/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test
index 2e73767..689e78a 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/mem-limit-broadcast-join.test
@@ -2,6 +2,8 @@ select * from tpch.nation n1
 join[broadcast]
 tpch.nation n2 on n1.n_regionkey=n2.n_regionkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 02:HASH JOIN [INNER JOIN, BROADCAST]

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test b/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
index 6270c11..da2e17f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/nested-collections.test
@@ -2,6 +2,8 @@
 select * from functional.allcomplextypes.int_array_col
 where item > 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.allcomplextypes.int_array_col]
    partitions=0/0 files=0 size=0B
    predicates: item > 10
@@ -10,6 +12,8 @@ where item > 10
 select * from functional.allcomplextypes.int_map_col
 where key = 'test' and value < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.allcomplextypes.int_map_col]
    partitions=0/0 files=0 size=0B
    predicates: value < 10, key = 'test'
@@ -18,6 +22,8 @@ where key = 'test' and value < 10
 select count(f21) from functional.allcomplextypes.complex_nested_struct_col.f2.f12
 where key = 'test'
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: count(f21)
 |
@@ -32,6 +38,8 @@ inner join functional.allcomplextypes.struct_map_col b
 on (a.f1 = b.f1)
 where a.f2 = 'test_a' and b.f2 = 'test_b'
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -58,6 +66,8 @@ select 1 from
 where c_nationkey = n_nationkey and s_nationkey = n_nationkey
   and c_comment = s_comment and n_comment = s_comment
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [INNER JOIN]
 |  hash predicates: c_nationkey = n_nationkey, s_comment = n_comment
 |  runtime filters: RF000 <- n_nationkey, RF001 <- n_comment
@@ -90,6 +100,8 @@ where c_nationkey = n_nationkey and s_nationkey = n_nationkey
 select a.id, b.item from functional.allcomplextypes a cross join a.int_array_col b
 where a.id < 10 and b.item % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [CROSS JOIN]
@@ -107,6 +119,8 @@ where a.id < 10 and b.item % 2 = 0
 select a.id from functional.allcomplextypes a left semi join a.int_array_col b
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT SEMI JOIN]
@@ -123,6 +137,8 @@ where a.id < 10
 select b.item from functional.allcomplextypes a right semi join a.int_array_col b
 where b.item % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT SEMI JOIN]
@@ -140,6 +156,8 @@ where b.item % 2 = 0
 select a.id from functional.allcomplextypes a left anti join a.int_array_col b
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT ANTI JOIN]
@@ -157,6 +175,8 @@ select a.id from functional.allcomplextypes a
 left anti join (select * from a.int_array_col) v
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT ANTI JOIN]
@@ -173,6 +193,8 @@ where a.id < 10
 select b.item from functional.allcomplextypes a right anti join a.int_array_col b
 where b.item % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT ANTI JOIN]
@@ -189,6 +211,8 @@ where b.item % 2 = 0
 select a.id, b.item from functional.allcomplextypes a left outer join a.int_array_col b
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT OUTER JOIN]
@@ -206,6 +230,8 @@ select a.id, v.item from functional.allcomplextypes a
 left outer join (select * from a.int_array_col) v
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT OUTER JOIN]
@@ -222,6 +248,8 @@ where a.id < 10
 select a.id, b.item from functional.allcomplextypes a right outer join a.int_array_col b
 where b.item % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT OUTER JOIN]
@@ -238,6 +266,8 @@ where b.item % 2 = 0
 select a.id, b.item from functional.allcomplextypes a full outer join a.int_array_col b
 where b.item % 2 = 0 and a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [FULL OUTER JOIN]
@@ -256,6 +286,8 @@ where b.item % 2 = 0 and a.id < 10
 select a.id, b.item from functional.allcomplextypes a, a.int_array_col b
 where a.id < 10 and b.item % 2 = 0 and a.id < b.item
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [INNER JOIN]
@@ -275,6 +307,8 @@ select a.id from functional.allcomplextypes a
   left semi join a.int_array_col b on (a.id < b.item and b.item % 2 = 0)
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT SEMI JOIN]
@@ -294,6 +328,8 @@ select b.item from functional.allcomplextypes a
   right semi join a.int_array_col b on (a.id < b.item and a.id < 10)
 where b.item % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT SEMI JOIN]
@@ -313,6 +349,8 @@ select a.id from functional.allcomplextypes a
   left anti join a.int_array_col b on (a.id < b.item and b.item % 2 = 0)
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT ANTI JOIN]
@@ -332,6 +370,8 @@ select b.item from functional.allcomplextypes a
   right anti join a.int_array_col b on (a.id < b.item and a.id < 10)
 where b.item % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT ANTI JOIN]
@@ -351,6 +391,8 @@ select a.id, b.item from functional.allcomplextypes a
   left outer join a.int_array_col b on (a.id < b.item and b.item % 2 = 0)
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT OUTER JOIN]
@@ -370,6 +412,8 @@ select a.id, b.item from functional.allcomplextypes a
   right outer join a.int_array_col b on (a.id < b.item and a.id < 10)
 where b.item % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT OUTER JOIN]
@@ -389,6 +433,8 @@ select a.id, b.item from functional.allcomplextypes a
   full outer join a.int_array_col b on (a.id < b.item and a.id < 10)
 where b.item % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [FULL OUTER JOIN]
@@ -408,6 +454,8 @@ select a.id, b.f1, b.f2 from functional.allcomplextypes a
   inner join a.struct_array_col b
 where a.id < 10 and b.f1 % 2 = 0 and b.f1 = a.id and b.f1 < a.year
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [INNER JOIN]
@@ -429,6 +477,8 @@ select a.id from functional.allcomplextypes a
     on (b.f1 % 2 = 0 and b.f1 = a.id and b.f1 < a.year)
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT SEMI JOIN]
@@ -449,6 +499,8 @@ select b.f1, b.f2 from functional.allcomplextypes a
     on (a.id < 10 and b.f1 = a.id and b.f1 < a.year)
 where b.f1 % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT SEMI JOIN]
@@ -469,6 +521,8 @@ select a.id from functional.allcomplextypes a
     on (b.f1 % 2 = 0 and b.f1 = a.id and b.f1 < a.year)
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT ANTI JOIN]
@@ -489,6 +543,8 @@ select b.f1, b.f2 from functional.allcomplextypes a
     on (a.id < 10 and b.f1 = a.id and b.f1 < a.year)
 where b.f1 % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT ANTI JOIN]
@@ -509,6 +565,8 @@ select a.id from functional.allcomplextypes a
     on (b.f1 % 2 = 0 and b.f1 = a.id and b.f1 < a.year)
 where a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT OUTER JOIN]
@@ -529,6 +587,8 @@ select b.f1, b.f2 from functional.allcomplextypes a
     on (a.id < 10 and b.f1 = a.id and b.f1 < a.year)
 where b.f1 % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [LEFT OUTER JOIN]
@@ -549,6 +609,8 @@ select b.f1, b.f2 from functional.allcomplextypes a
     on (b.f1 = a.id and b.f1 < a.year)
 where a.id < 10 and b.f1 % 2 = 0
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [FULL OUTER JOIN]
@@ -572,6 +634,8 @@ select 1 from functional.allcomplextypes a
   left outer join a.struct_array_col d on (a.month < 4 or d.f1 > 5)
   inner join a.struct_map_col e on (e.f1 = a.id and e.f2 = 'test')
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--10:HASH JOIN [INNER JOIN]
@@ -612,6 +676,8 @@ inner join functional.alltypes d on (b.id = d.id)
 inner join a.struct_array_col e
 where e.f1 < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 11:HASH JOIN [INNER JOIN]
 |  hash predicates: d.id = b.id
 |  runtime filters: RF000 <- b.id
@@ -656,6 +722,8 @@ inner join a.struct_array_col e
 right anti join functional.alltypessmall c on (b.int_col = c.int_col and e.f1 < 10)
 inner join functional.alltypes d on (b.id = d.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 11:HASH JOIN [INNER JOIN]
 |  hash predicates: d.id = b.id
 |
@@ -697,6 +765,8 @@ inner join a.int_map_col d
 left semi join functional.alltypes e on (d.value = e.id)
 where b.item < 10 and c.int_col > 30
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:HASH JOIN [RIGHT SEMI JOIN]
 |  hash predicates: e.id = d.value
 |  runtime filters: RF000 <- d.value
@@ -745,6 +815,8 @@ inner join a.int_map_col d
 right anti join functional.alltypestiny e on (d.value = e.id)
 where b.item < 10 and c.int_col > 30
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:HASH JOIN [RIGHT ANTI JOIN]
 |  hash predicates: d.value = e.id
 |
@@ -785,6 +857,8 @@ from functional.allcomplextypes a,
   (select count(*) cnt from a.int_array_col) v
 where v.cnt < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--05:NESTED LOOP JOIN [CROSS JOIN]
@@ -806,6 +880,8 @@ from functional.allcomplextypes a,
   (select f1, count(*) cnt from a.struct_array_col group by f1) v
 where v.cnt < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--05:NESTED LOOP JOIN [CROSS JOIN]
@@ -828,6 +904,8 @@ from functional.allcomplextypes a,
   (select * from a.struct_array_col order by f1 limit 10) v
 where v.f2 = 'test'
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--06:NESTED LOOP JOIN [CROSS JOIN]
@@ -851,6 +929,8 @@ from functional.allcomplextypes a,
   (select key, row_number() over (order by key) rnum from a.int_map_col) v
 where v.key != 'bad'
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--07:NESTED LOOP JOIN [CROSS JOIN]
@@ -877,6 +957,8 @@ where v.key != 'bad'
 select a.id from functional.allcomplextypes a
 where id < (select avg(item) from a.int_array_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--05:NESTED LOOP JOIN [RIGHT SEMI JOIN]
@@ -896,6 +978,8 @@ where id < (select avg(item) from a.int_array_col)
 select a.id from functional.allcomplextypes a
 where exists (select item from a.int_array_col where item > 100)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT SEMI JOIN]
@@ -912,6 +996,8 @@ where exists (select item from a.int_array_col where item > 100)
 select a.id from functional.allcomplextypes a
 where not exists (select item from a.int_array_col)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT ANTI JOIN]
@@ -928,6 +1014,8 @@ where not exists (select item from a.int_array_col)
 select a.id from functional.allcomplextypes a
 where exists (select m.key from a.struct_map_col m where a.id < m.f1)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT SEMI JOIN]
@@ -945,6 +1033,8 @@ where exists (select m.key from a.struct_map_col m where a.id < m.f1)
 select a.id from functional.allcomplextypes a
 where not exists (select c.f2 from a.struct_array_col c where a.id < c.f1)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT ANTI JOIN]
@@ -964,6 +1054,8 @@ select a.id
 from functional.allcomplextypes a
 where id in (select b.item from a.int_array_col b where a.year < b.item)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT SEMI JOIN]
@@ -984,6 +1076,8 @@ select a.id
 from functional.allcomplextypes a
 where id not in (select b.item from a.int_array_col b)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
@@ -1004,6 +1098,8 @@ select a.id
 from functional.allcomplextypes a
 where id not in (select b.item from a.int_array_col b where a.year < b.item)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
@@ -1028,6 +1124,8 @@ cross join
    from a.struct_array_col x inner join b.struct_map_col y
    on x.f1 = y.f1) v
 ---- PLAN
+PLAN-ROOT SINK
+|
 17:SUBPLAN
 |
 |--15:NESTED LOOP JOIN [CROSS JOIN]
@@ -1085,6 +1183,8 @@ cross join
    on x.f1 = y.f1) v
 where b.id = d.value
 ---- PLAN
+PLAN-ROOT SINK
+|
 17:SUBPLAN
 |
 |--15:NESTED LOOP JOIN [CROSS JOIN]
@@ -1145,6 +1245,8 @@ cross join
    from a.struct_array_col x inner join b.struct_map_col y
    on x.f1 = y.f1) v
 ---- PLAN
+PLAN-ROOT SINK
+|
 15:SUBPLAN
 |
 |--13:NESTED LOOP JOIN [CROSS JOIN]
@@ -1192,6 +1294,8 @@ 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
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |  limit: 10
 |
@@ -1222,6 +1326,8 @@ cross join
   (select m1.key from a.map_map_col m1,
    (select m2.key from m1.value m2) v1) v2
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--08:NESTED LOOP JOIN [CROSS JOIN]
@@ -1249,6 +1355,8 @@ inner join functional.allcomplextypes b on (a.id = b.id)
 cross join (select count(*) c from a.int_map_col) v1
 cross join (select avg(item) a from b.int_array_col) v2
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:SUBPLAN
 |
 |--08:NESTED LOOP JOIN [CROSS JOIN]
@@ -1288,6 +1396,8 @@ where c.c_custkey = o.o_orderkey and c.c_custkey = o.o_shippriority
 # redundant predicates
   and o.o_orderkey = l.l_partkey and o.o_shippriority = l.l_suppkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--08:NESTED LOOP JOIN [INNER JOIN]
@@ -1327,6 +1437,8 @@ where a.item between 10 and 20
   and v2.key = 'test2'
   and v2.x = 'test3'
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--12:NESTED LOOP JOIN [INNER JOIN]
@@ -1376,6 +1488,8 @@ where s.s_suppkey not in
  inner join s.s_partsupps ps3
    on ps2.ps_comment = ps3.ps_comment)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--08:HASH JOIN [NULL AWARE LEFT ANTI JOIN]
@@ -1404,6 +1518,8 @@ left outer join functional.allcomplextypes t2 ON (t1.id = t2.id)
 # The subplan for this table ref must come after the outer join of t1 and t2.
 inner join t2.int_array_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [CROSS JOIN]
@@ -1426,6 +1542,8 @@ select a from functional.allcomplextypes t,
   (select count(*) over(partition by key) a
    from t.int_map_col group by key) v
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--07:NESTED LOOP JOIN [CROSS JOIN]
@@ -1454,6 +1572,8 @@ left outer join functional.allcomplextypes t2 on (t1.id = t2.id)
 inner join t1.map_map_col m1
 inner join m1.value m2
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:SUBPLAN
 |
 |--08:SUBPLAN
@@ -1494,6 +1614,8 @@ where c.c_custkey in
  left outer join c.c_orders o3 on o3.pos = o2.pos
  where c.c_custkey = o2.pos)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--12:HASH JOIN [RIGHT SEMI JOIN]
@@ -1538,6 +1660,8 @@ where c.c_custkey in
  inner join c.c_orders o3 on o3.pos = o2.pos
  where c.c_custkey = o2.pos)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--12:HASH JOIN [RIGHT SEMI JOIN]
@@ -1580,6 +1704,8 @@ where c.c_custkey in
  left outer join o2.o_lineitems l
  where c.c_custkey = o2.pos)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--12:HASH JOIN [RIGHT SEMI JOIN]
@@ -1616,6 +1742,8 @@ where c.c_custkey in
 select straight_join id from functional.allcomplextypes t1 left outer join t1.int_array_col t2
 where t1.id = t2.pos and t1.int_struct_col.f1 = 10 and t2.item = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:HASH JOIN [LEFT OUTER JOIN]
@@ -1637,6 +1765,8 @@ where t1.id = t2.pos and t1.int_struct_col.f1 = 10 and t2.item = 1
 select straight_join id from functional.allcomplextypes t1 right outer join t1.int_array_col t2
 where t1.id = t2.pos and t1.int_struct_col.f1 = 10 and t2.item = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:HASH JOIN [RIGHT OUTER JOIN]
@@ -1658,6 +1788,8 @@ where t1.id = t2.pos and t1.int_struct_col.f1 = 10 and t2.item = 1
 select id from functional.allcomplextypes t1 full outer join t1.int_array_col t2
 where t1.id = t2.pos and t1.int_struct_col.f1 = 10 and t2.item = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [FULL OUTER JOIN]
@@ -1679,6 +1811,8 @@ select id from functional.allcomplextypes t1 left outer join
   (select pos, item from t1.int_array_col t2) v
  where t1.id > v.pos and v.item = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT OUTER JOIN]
@@ -1698,6 +1832,8 @@ left outer join c.c_orders o
 # Has an ordering dependency on c and o
 inner join o.o_lineitems
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--08:SUBPLAN
@@ -1724,6 +1860,8 @@ inner join c1.c_orders
 right outer join tpch_nested_parquet.customer c2
   on c1.c_custkey = c2.c_custkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:HASH JOIN [RIGHT OUTER JOIN]
 |  hash predicates: c1.c_custkey = c2.c_custkey
 |  runtime filters: RF000 <- c2.c_custkey
@@ -1751,6 +1889,8 @@ full outer join tpch_nested_parquet.customer c2
 inner join c1.c_orders o1
 left semi join c2.c_orders o2
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:SUBPLAN
 |
 |--06:NESTED LOOP JOIN [LEFT SEMI JOIN]
@@ -1778,6 +1918,8 @@ select * from tpch_nested_parquet.customer c
 left outer join c.c_orders o
 where o.o_orderkey is null and o.o_orderstatus <=> o_orderpriority
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:SUBPLAN
 |
 |--04:NESTED LOOP JOIN [RIGHT OUTER JOIN]
@@ -1799,6 +1941,8 @@ left join t2.c_orders t4
 inner join tpch_nested_parquet.region t5 on t5.r_regionkey = t2.c_custkey
 left join t4.item.o_lineitems t6 on t6.item.l_returnflag = t4.item.o_orderpriority
 ---- PLAN
+PLAN-ROOT SINK
+|
 14:SUBPLAN
 |
 |--12:SUBPLAN

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/nested-loop-join.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/nested-loop-join.test b/testdata/workloads/functional-planner/queries/PlannerTest/nested-loop-join.test
index c7f9830..fecec7f 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/nested-loop-join.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/nested-loop-join.test
@@ -6,6 +6,8 @@ right outer join functional.alltypesagg c
   on a.smallint_col >= c.smallint_col
 where a.id < 10 and c.bigint_col = 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:NESTED LOOP JOIN [RIGHT OUTER JOIN]
 |  join predicates: a.smallint_col >= c.smallint_col
 |  predicates: a.id < 10
@@ -34,6 +36,8 @@ right semi join functional.alltypesagg d
   on c.tinyint_col < d.bigint_col
 where d.bigint_col < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:NESTED LOOP JOIN [RIGHT SEMI JOIN]
 |  join predicates: c.tinyint_col < d.bigint_col
 |
@@ -62,6 +66,8 @@ full outer join functional.alltypes d
   on c.int_col > d.int_col
 where a.bigint_col != c.bigint_col and a.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:NESTED LOOP JOIN [FULL OUTER JOIN]
 |  join predicates: c.int_col > d.int_col
 |  predicates: a.bigint_col != c.bigint_col, a.id < 10
@@ -93,6 +99,8 @@ from functional.alltypestiny a right anti join functional.alltypessmall b
   on a.id < b.id
 where b.int_col = 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -116,6 +124,8 @@ right semi join functional.alltypes d on c.tinyint_col < d.tinyint_col
 right anti join functional.alltypesnopart e on d.tinyint_col > e.tinyint_col
 where e.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -158,6 +168,8 @@ right semi join functional.alltypes d on c.tinyint_col < d.tinyint_col
 right anti join functional.alltypesnopart e on d.tinyint_col > e.tinyint_col
 where e.id < 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 15:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |


[2/7] incubator-impala git commit: IMPALA-2905: Handle coordinator fragment lifecycle like all others

Posted by he...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
index 4de9722..8144524 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpcds-all.test
@@ -28,6 +28,8 @@ order by
   brand_id
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=100]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
 |
@@ -56,6 +58,8 @@ limit 100
    predicates: dt.d_moy = 12, (dt.d_date_sk BETWEEN 2451149 AND 2451179 OR dt.d_date_sk BETWEEN 2451514 AND 2451544 OR dt.d_date_sk BETWEEN 2451880 AND 2451910 OR dt.d_date_sk BETWEEN 2452245 AND 2452275 OR dt.d_date_sk BETWEEN 2452610 AND 2452640)
    runtime filters: RF000 -> dt.d_date_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
 |  limit: 100
@@ -98,6 +102,8 @@ limit 100
    predicates: dt.d_moy = 12, (dt.d_date_sk BETWEEN 2451149 AND 2451179 OR dt.d_date_sk BETWEEN 2451514 AND 2451544 OR dt.d_date_sk BETWEEN 2451880 AND 2451910 OR dt.d_date_sk BETWEEN 2452245 AND 2452275 OR dt.d_date_sk BETWEEN 2452610 AND 2452640)
    runtime filters: RF000 -> dt.d_date_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
 |  limit: 100
@@ -178,6 +184,8 @@ order by
   i_item_id
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC
 |
@@ -220,6 +228,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_item_sk, RF001 -> ss_sold_date_sk, RF002 -> ss_cdemo_sk, RF003 -> ss_promo_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
 |  limit: 100
@@ -282,6 +292,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_item_sk, RF001 -> ss_sold_date_sk, RF002 -> ss_cdemo_sk, RF003 -> ss_promo_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC
 |  limit: 100
@@ -430,6 +442,8 @@ order by
   s_store_name
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 11:TOP-N [LIMIT=100]
 |  order by: s_store_name ASC
 |
@@ -479,6 +493,8 @@ limit 100
    partitions=7/120 files=7 size=1.02MB
    runtime filters: RF001 -> store_sales.ss_store_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC
 |  limit: 100
@@ -552,6 +568,8 @@ limit 100
    partitions=7/120 files=7 size=1.02MB
    runtime filters: RF001 -> store_sales.ss_store_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC
 |  limit: 100
@@ -679,6 +697,8 @@ order by
   i_manufact
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:TOP-N [LIMIT=100]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC
 |
@@ -730,6 +750,8 @@ limit 100
    partitions=1/1 files=1 size=5.25MB
    runtime filters: RF004 -> ca_address_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC
 |  limit: 100
@@ -801,6 +823,8 @@ limit 100
    partitions=1/1 files=1 size=5.25MB
    runtime filters: RF004 -> ca_address_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand ASC, i_brand_id ASC, i_manufact_id ASC, i_manufact ASC
 |  limit: 100
@@ -924,6 +948,8 @@ order by
   s_state
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=100]
 |  order by: i_item_id ASC, s_state ASC
 |
@@ -966,6 +992,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_item_sk, RF001 -> ss_store_sk, RF002 -> ss_sold_date_sk, RF003 -> ss_cdemo_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, s_state ASC
 |  limit: 100
@@ -1028,6 +1056,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_item_sk, RF001 -> ss_store_sk, RF002 -> ss_sold_date_sk, RF003 -> ss_cdemo_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_item_id ASC, s_state ASC
 |  limit: 100
@@ -1151,6 +1181,8 @@ order by
   c_preferred_cust_flag desc
 limit 100000
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=100000]
 |  order by: c_last_name ASC, c_first_name ASC, c_salutation ASC, c_preferred_cust_flag DESC
 |
@@ -1195,6 +1227,8 @@ limit 100000
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> c_customer_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, c_salutation ASC, c_preferred_cust_flag DESC
 |  limit: 100000
@@ -1257,6 +1291,8 @@ limit 100000
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> c_customer_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, c_salutation ASC, c_preferred_cust_flag DESC
 |  limit: 100000
@@ -1362,6 +1398,8 @@ order by
   item.i_category
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=100]
 |  order by: sum(ss_ext_sales_price) DESC, dt.d_year ASC, item.i_category_id ASC, item.i_category ASC
 |
@@ -1389,6 +1427,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> store_sales.ss_sold_date_sk, RF001 -> store_sales.ss_item_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, dt.d_year ASC, item.i_category_id ASC, item.i_category ASC
 |  limit: 100
@@ -1430,6 +1470,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> store_sales.ss_sold_date_sk, RF001 -> store_sales.ss_item_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, dt.d_year ASC, item.i_category_id ASC, item.i_category ASC
 |  limit: 100
@@ -1514,6 +1556,8 @@ order by
   sat_sales
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=100]
 |  order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC
 |
@@ -1541,6 +1585,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_store_sk, RF001 -> ss_sold_date_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC
 |  limit: 100
@@ -1582,6 +1628,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_store_sk, RF001 -> ss_sold_date_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, s_store_id ASC, sum(CASE WHEN (d_day_name = 'Sunday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Monday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Tuesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Wednesday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Thursday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Friday') THEN ss_sales_price ELSE NULL END) ASC, sum(CASE WHEN (d_day_name = 'Saturday') THEN ss_sales_price ELSE NULL END) ASC
 |  limit: 100
@@ -1710,6 +1758,8 @@ order by
   ss_ticket_number
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 14:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, c_first_name ASC, ca_city ASC, bought_city ASC, ss_ticket_number ASC
 |
@@ -1770,6 +1820,8 @@ limit 100
    partitions=1/1 files=1 size=5.25MB
    runtime filters: RF000 -> current_addr.ca_address_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, ca_city ASC, bought_city ASC, ss_ticket_number ASC
 |  limit: 100
@@ -1852,6 +1904,8 @@ limit 100
    partitions=1/1 files=1 size=5.25MB
    runtime filters: RF000 -> current_addr.ca_address_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, ca_city ASC, bought_city ASC, ss_ticket_number ASC
 |  limit: 100
@@ -1984,6 +2038,8 @@ order by
   brand_id
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=100]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
 |
@@ -2011,6 +2067,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> store_sales.ss_sold_date_sk, RF001 -> store_sales.ss_item_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
 |  limit: 100
@@ -2052,6 +2110,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> store_sales.ss_sold_date_sk, RF001 -> store_sales.ss_item_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: dt.d_year ASC, sum(ss_ext_sales_price) DESC, item.i_brand_id ASC
 |  limit: 100
@@ -2133,6 +2193,8 @@ order by
   i_manufact_id
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:TOP-N [LIMIT=100]
 |  order by: sum_sales ASC, i_manufact_id ASC
 |
@@ -2168,6 +2230,8 @@ limit 100
    predicates: d_month_seq IN (1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11)
    runtime filters: RF001 -> d_date_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales ASC, i_manufact_id ASC
 |  limit: 100
@@ -2219,6 +2283,8 @@ limit 100
    predicates: d_month_seq IN (1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11)
    runtime filters: RF001 -> d_date_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales ASC, i_manufact_id ASC
 |  limit: 100
@@ -2305,6 +2371,8 @@ order by
   i_brand_id
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=100]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC
 |
@@ -2332,6 +2400,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_sold_date_sk, RF001 -> ss_item_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC
 |  limit: 100
@@ -2373,6 +2443,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_sold_date_sk, RF001 -> ss_item_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(ss_ext_sales_price) DESC, i_brand_id ASC
 |  limit: 100
@@ -2505,6 +2577,8 @@ order by
   d_week_seq1
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 17:TOP-N [LIMIT=100]
 |  order by: s_store_name1 ASC, s_store_id1 ASC, d_week_seq1 ASC
 |
@@ -2576,6 +2650,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF002 -> tpcds.store_sales.ss_store_sk, RF004 -> ss_sold_date_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 32:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name1 ASC, s_store_id1 ASC, d_week_seq1 ASC
 |  limit: 100
@@ -2683,6 +2759,8 @@ limit 100
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF002 -> tpcds.store_sales.ss_store_sk, RF004 -> ss_sold_date_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 32:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name1 ASC, s_store_id1 ASC, d_week_seq1 ASC
 |  limit: 100
@@ -2855,6 +2933,8 @@ order by
   sum_sales
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 11:TOP-N [LIMIT=100]
 |  order by: i_manager_id ASC, avg_monthly_sales ASC, sum_sales ASC
 |
@@ -2900,6 +2980,8 @@ limit 100
    predicates: tpcds.date_dim.d_date_sk <= 2452275, tpcds.date_dim.d_date_sk >= 2451911, d_month_seq IN (1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11)
    runtime filters: RF001 -> d_date_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_manager_id ASC, avg_monthly_sales ASC, sum_sales ASC
 |  limit: 100
@@ -2961,6 +3043,8 @@ limit 100
    predicates: tpcds.date_dim.d_date_sk <= 2452275, tpcds.date_dim.d_date_sk >= 2451911, d_month_seq IN (1212, 1212 + 1, 1212 + 2, 1212 + 3, 1212 + 4, 1212 + 5, 1212 + 6, 1212 + 7, 1212 + 8, 1212 + 9, 1212 + 10, 1212 + 11)
    runtime filters: RF001 -> d_date_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_manager_id ASC, avg_monthly_sales ASC, sum_sales ASC
 |  limit: 100
@@ -3090,6 +3174,8 @@ order by
   i_item_desc
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 14:TOP-N [LIMIT=100]
 |  order by: s_store_name ASC, i_item_desc ASC
 |
@@ -3150,6 +3236,8 @@ limit 100
    partitions=1/1 files=1 size=4.82MB
    runtime filters: RF000 -> i_item_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 26:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, i_item_desc ASC
 |  limit: 100
@@ -3242,6 +3330,8 @@ limit 100
    partitions=1/1 files=1 size=4.82MB
    runtime filters: RF000 -> i_item_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 26:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_store_name ASC, i_item_desc ASC
 |  limit: 100
@@ -3404,6 +3494,8 @@ order by
   ss_ticket_number
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 14:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, ss_ticket_number ASC
 |
@@ -3462,6 +3554,8 @@ limit 100
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> customer.c_current_addr_sk, RF001 -> c_customer_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, ss_ticket_number ASC
 |  limit: 100
@@ -3542,6 +3636,8 @@ limit 100
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> customer.c_current_addr_sk, RF001 -> c_customer_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, ss_ticket_number ASC
 |  limit: 100
@@ -3685,6 +3781,8 @@ order by
   cnt desc
 limit 1000
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=1000]
 |  order by: cnt DESC
 |
@@ -3728,6 +3826,8 @@ limit 1000
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> c_customer_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cnt DESC
 |  limit: 1000
@@ -3791,6 +3891,8 @@ limit 1000
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> c_customer_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cnt DESC
 |  limit: 1000
@@ -3914,6 +4016,8 @@ order by
   profit
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=100]
 |  order by: c_last_name ASC, c_first_name ASC, substr(s_city, 1, 30) ASC, profit ASC
 |
@@ -3957,6 +4061,8 @@ limit 100
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> c_customer_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, substr(s_city, 1, 30) ASC, profit ASC
 |  limit: 100
@@ -4020,6 +4126,8 @@ limit 100
    partitions=1/1 files=1 size=12.60MB
    runtime filters: RF000 -> c_customer_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: c_last_name ASC, c_first_name ASC, substr(s_city, 1, 30) ASC, profit ASC
 |  limit: 100
@@ -4129,6 +4237,8 @@ where case when (avg_monthly_sales <> 0) then (abs(sum_sales - avg_monthly_sales
 order by sum_sales - avg_monthly_sales, s_store_name
 limit 100) tmp2
 ---- PLAN
+PLAN-ROOT SINK
+|
 11:TOP-N [LIMIT=100]
 |  order by: sum_sales - avg_monthly_sales ASC, s_store_name ASC
 |
@@ -4173,6 +4283,8 @@ limit 100) tmp2
    partitions=24/120 files=24 size=4.16MB
    runtime filters: RF000 -> ss_store_sk, RF001 -> ss_sold_date_sk, RF002 -> ss_item_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC, s_store_name ASC
 |  limit: 100
@@ -4233,6 +4345,8 @@ limit 100) tmp2
    partitions=24/120 files=24 size=4.16MB
    runtime filters: RF000 -> ss_store_sk, RF001 -> ss_sold_date_sk, RF002 -> ss_item_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC, s_store_name ASC
 |  limit: 100
@@ -4321,6 +4435,8 @@ WHERE
   AND hd.hd_dep_count = 5
   AND s.s_store_name = 'ese'
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -4352,6 +4468,8 @@ WHERE
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss.ss_store_sk, RF001 -> ss.ss_hdemo_sk, RF002 -> ss.ss_sold_time_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 12:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |
@@ -4394,6 +4512,8 @@ WHERE
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss.ss_store_sk, RF001 -> ss.ss_hdemo_sk, RF002 -> ss.ss_sold_time_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 12:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |
@@ -4480,6 +4600,8 @@ order by
   revenueratio
 limit 1000
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:TOP-N [LIMIT=1000]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) ASC
 |
@@ -4515,6 +4637,8 @@ limit 1000
    predicates: tpcds.date_dim.d_date_sk <= 2451941, tpcds.date_dim.d_date_sk >= 2451911, d_date <= '2001-01-31', d_date >= '2001-01-01'
    runtime filters: RF000 -> d_date_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) ASC
 |  limit: 1000
@@ -4564,6 +4688,8 @@ limit 1000
    predicates: tpcds.date_dim.d_date_sk <= 2451941, tpcds.date_dim.d_date_sk >= 2451911, d_date <= '2001-01-31', d_date >= '2001-01-01'
    runtime filters: RF000 -> d_date_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: i_category ASC, i_class ASC, i_item_id ASC, i_item_desc ASC, sum(ss_ext_sales_price) * 100 / sum(sum(ss_ext_sales_price)) ASC
 |  limit: 1000
@@ -4648,6 +4774,8 @@ select * from (
  having count(*) >= 10
  order by cnt limit 100) as t
 ---- PLAN
+PLAN-ROOT SINK
+|
 16:TOP-N [LIMIT=100]
 |  order by: count(*) ASC
 |
@@ -4715,6 +4843,8 @@ select * from (
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF002 -> s.ss_item_sk, RF003 -> s.ss_sold_date_sk, RF005 -> s.ss_customer_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 31:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) ASC
 |  limit: 100
@@ -4820,6 +4950,8 @@ select * from (
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF002 -> s.ss_item_sk, RF003 -> s.ss_sold_date_sk, RF005 -> s.ss_customer_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 31:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) ASC
 |  limit: 100
@@ -5000,6 +5132,8 @@ with v1 as (
  limit 100
  ) as v3
 ---- PLAN
+PLAN-ROOT SINK
+|
 35:TOP-N [LIMIT=100]
 |  order by: sum_sales - avg_monthly_sales ASC, d_year ASC
 |
@@ -5136,6 +5270,8 @@ with v1 as (
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_store_sk, RF001 -> ss_sold_date_sk, RF002 -> ss_item_sk
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 53:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC, d_year ASC
 |  limit: 100
@@ -5316,6 +5452,8 @@ with v1 as (
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF000 -> ss_store_sk, RF001 -> ss_sold_date_sk, RF002 -> ss_item_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 53:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum_sales - avg_monthly_sales ASC, d_year ASC
 |  limit: 100
@@ -5582,6 +5720,8 @@ from
    and   d_moy  = 11) all_sales
 order by promotions, total
 ---- PLAN
+PLAN-ROOT SINK
+|
 27:SORT
 |  order by: promotions ASC, total ASC
 |
@@ -5688,6 +5828,8 @@ order by promotions, total
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF005 -> ss_item_sk, RF004 -> ss_promo_sk, RF003 -> ss_sold_date_sk, RF001 -> ss_customer_sk, RF002 -> ss_store_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 27:SORT
 |  order by: promotions ASC, total ASC
 |
@@ -5972,6 +6114,8 @@ from
           (household_demographics.hd_dep_count = 2 and household_demographics.hd_vehicle_count<=2+2))
      and store.s_store_name = 'ese') s8
 ---- PLAN
+PLAN-ROOT SINK
+|
 70:NESTED LOOP JOIN [CROSS JOIN]
 |
 |--63:AGGREGATE [FINALIZE]
@@ -6220,6 +6364,8 @@ from
    partitions=120/120 files=120 size=21.31MB
    runtime filters: RF002 -> ss_sold_time_sk, RF001 -> ss_hdemo_sk
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 70:NESTED LOOP JOIN [CROSS JOIN, BROADCAST]
 |  join table id: 00
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
index fc6f4e2..d883b74 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-all.test
@@ -22,6 +22,8 @@ order by
   l_returnflag,
   l_linestatus
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: l_returnflag ASC, l_linestatus ASC
 |
@@ -33,6 +35,8 @@ order by
    partitions=1/1 files=1 size=718.94MB
    predicates: l_shipdate <= '1998-09-02'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_returnflag ASC, l_linestatus ASC
 |
@@ -53,6 +57,8 @@ order by
    partitions=1/1 files=1 size=718.94MB
    predicates: l_shipdate <= '1998-09-02'
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_returnflag ASC, l_linestatus ASC
 |
@@ -120,6 +126,8 @@ order by
   p_partkey
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 18:TOP-N [LIMIT=100]
 |  order by: s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC
 |
@@ -195,6 +203,8 @@ limit 100
    partitions=1/1 files=1 size=112.71MB
    runtime filters: RF000 -> tpch.partsupp.ps_partkey, RF004 -> ps_suppkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC
 |  limit: 100
@@ -298,6 +308,8 @@ limit 100
    partitions=1/1 files=1 size=112.71MB
    runtime filters: RF000 -> tpch.partsupp.ps_partkey, RF004 -> ps_suppkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 30:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC
 |  limit: 100
@@ -459,6 +471,8 @@ order by
   o_orderdate
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=10]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC, o_orderdate ASC
 |
@@ -488,6 +502,8 @@ limit 10
    predicates: l_shipdate > '1995-03-15'
    runtime filters: RF001 -> l_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC, o_orderdate ASC
 |  limit: 10
@@ -531,6 +547,8 @@ limit 10
    predicates: l_shipdate > '1995-03-15'
    runtime filters: RF001 -> l_orderkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC, o_orderdate ASC
 |  limit: 10
@@ -606,6 +624,8 @@ group by
 order by
   o_orderpriority
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: o_orderpriority ASC
 |
@@ -626,6 +646,8 @@ order by
    predicates: l_commitdate < l_receiptdate
    runtime filters: RF000 -> l_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_orderpriority ASC
 |
@@ -659,6 +681,8 @@ order by
    predicates: l_commitdate < l_receiptdate
    runtime filters: RF000 -> l_orderkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_orderpriority ASC
 |
@@ -723,6 +747,8 @@ group by
 order by
   revenue desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -775,6 +801,8 @@ order by
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF003 -> l_suppkey, RF005 -> l_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -846,6 +874,8 @@ order by
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF003 -> l_suppkey, RF005 -> l_orderkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 20:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -949,6 +979,8 @@ where
   and l_discount between 0.05 and 0.07
   and l_quantity < 24
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice * l_discount)
 |
@@ -956,6 +988,8 @@ where
    partitions=1/1 files=1 size=718.94MB
    predicates: l_discount <= 0.07, l_discount >= 0.05, l_quantity < 24, l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice * l_discount)
 |
@@ -968,6 +1002,8 @@ where
    partitions=1/1 files=1 size=718.94MB
    predicates: l_discount <= 0.07, l_discount >= 0.05, l_quantity < 24, l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice * l_discount)
 |
@@ -1021,6 +1057,8 @@ order by
   cust_nation,
   l_year
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: supp_nation ASC, cust_nation ASC, l_year ASC
 |
@@ -1072,6 +1110,8 @@ order by
    predicates: l_shipdate <= '1996-12-31', l_shipdate >= '1995-01-01'
    runtime filters: RF003 -> l_suppkey, RF004 -> l_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 21:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: supp_nation ASC, cust_nation ASC, l_year ASC
 |
@@ -1144,6 +1184,8 @@ order by
    predicates: l_shipdate <= '1996-12-31', l_shipdate >= '1995-01-01'
    runtime filters: RF003 -> l_suppkey, RF004 -> l_orderkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 21:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: supp_nation ASC, cust_nation ASC, l_year ASC
 |
@@ -1276,6 +1318,8 @@ group by
 order by
   o_year
 ---- PLAN
+PLAN-ROOT SINK
+|
 16:SORT
 |  order by: o_year ASC
 |
@@ -1343,6 +1387,8 @@ order by
    partitions=1/1 files=1 size=23.08MB
    runtime filters: RF002 -> c_nationkey, RF003 -> c_custkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_year ASC
 |
@@ -1437,6 +1483,8 @@ order by
    partitions=1/1 files=1 size=23.08MB
    runtime filters: RF002 -> c_nationkey, RF003 -> c_custkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 28:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_year ASC
 |
@@ -1593,6 +1641,8 @@ order by
   nation,
   o_year desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: nation ASC, o_year DESC
 |
@@ -1642,6 +1692,8 @@ order by
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF001 -> l_partkey, RF002 -> l_suppkey, RF003 -> l_suppkey, RF004 -> l_orderkey, RF005 -> l_partkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 21:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: nation ASC, o_year DESC
 |
@@ -1712,6 +1764,8 @@ order by
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF001 -> l_partkey, RF002 -> l_suppkey, RF003 -> l_suppkey, RF004 -> l_orderkey, RF005 -> l_partkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 21:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: nation ASC, o_year DESC
 |
@@ -1838,6 +1892,8 @@ order by
   revenue desc
 limit 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:TOP-N [LIMIT=20]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -1873,6 +1929,8 @@ limit 20
    partitions=1/1 files=1 size=23.08MB
    runtime filters: RF000 -> c_nationkey, RF001 -> c_custkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |  limit: 20
@@ -1924,6 +1982,8 @@ limit 20
    partitions=1/1 files=1 size=23.08MB
    runtime filters: RF000 -> c_nationkey, RF001 -> c_custkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 14:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |  limit: 20
@@ -2023,6 +2083,8 @@ where
 order by
   value desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 13:SORT
 |  order by: value DESC
 |
@@ -2076,6 +2138,8 @@ order by
    partitions=1/1 files=1 size=112.71MB
    runtime filters: RF001 -> ps_suppkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: value DESC
 |
@@ -2153,6 +2217,8 @@ order by
    partitions=1/1 files=1 size=112.71MB
    runtime filters: RF001 -> ps_suppkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: value DESC
 |
@@ -2281,6 +2347,8 @@ group by
 order by
   l_shipmode
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: l_shipmode ASC
 |
@@ -2300,6 +2368,8 @@ order by
    partitions=1/1 files=1 size=162.56MB
    runtime filters: RF000 -> o_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_shipmode ASC
 |
@@ -2332,6 +2402,8 @@ order by
    partitions=1/1 files=1 size=162.56MB
    runtime filters: RF000 -> o_orderkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_shipmode ASC
 |
@@ -2391,6 +2463,8 @@ order by
   custdist desc,
   c_count desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:SORT
 |  order by: count(*) DESC, c_count DESC
 |
@@ -2414,6 +2488,8 @@ order by
    predicates: NOT o_comment LIKE '%special%requests%'
    runtime filters: RF000 -> o_custkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC, c_count DESC
 |
@@ -2450,6 +2526,8 @@ order by
    predicates: NOT o_comment LIKE '%special%requests%'
    runtime filters: RF000 -> o_custkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 10:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC, c_count DESC
 |
@@ -2506,6 +2584,8 @@ where
   and l_shipdate >= '1995-09-01'
   and l_shipdate < '1995-10-01'
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0.0 END), sum(l_extendedprice * (1 - l_discount))
 |
@@ -2521,6 +2601,8 @@ where
    predicates: l_shipdate < '1995-10-01', l_shipdate >= '1995-09-01'
    runtime filters: RF000 -> l_partkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:AGGREGATE [FINALIZE]
 |  output: sum:merge(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0.0 END), sum:merge(l_extendedprice * (1 - l_discount))
 |
@@ -2545,6 +2627,8 @@ where
    predicates: l_shipdate < '1995-10-01', l_shipdate >= '1995-09-01'
    runtime filters: RF000 -> l_partkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 07:AGGREGATE [FINALIZE]
 |  output: sum:merge(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0.0 END), sum:merge(l_extendedprice * (1 - l_discount))
 |
@@ -2606,6 +2690,8 @@ where
 order by
   s_suppkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:SORT
 |  order by: s_suppkey ASC
 |
@@ -2639,6 +2725,8 @@ order by
    partitions=1/1 files=1 size=1.33MB
    runtime filters: RF000 -> s_suppkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_suppkey ASC
 |
@@ -2696,6 +2784,8 @@ order by
    partitions=1/1 files=1 size=1.33MB
    runtime filters: RF000 -> s_suppkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_suppkey ASC
 |
@@ -2794,6 +2884,8 @@ order by
   p_type,
   p_size
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
 |
@@ -2823,6 +2915,8 @@ order by
    partitions=1/1 files=1 size=112.71MB
    runtime filters: RF000 -> ps_partkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 12:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
 |
@@ -2864,6 +2958,8 @@ order by
    partitions=1/1 files=1 size=112.71MB
    runtime filters: RF000 -> ps_partkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 12:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
 |
@@ -2933,6 +3029,8 @@ where
       l_partkey = p_partkey
   )
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice)
 |
@@ -2961,6 +3059,8 @@ where
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000 -> tpch.lineitem.l_partkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 12:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice)
 |
@@ -3004,6 +3104,8 @@ where
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000 -> tpch.lineitem.l_partkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 12:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice)
 |
@@ -3092,6 +3194,8 @@ order by
   o_orderdate
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:TOP-N [LIMIT=100]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |
@@ -3130,6 +3234,8 @@ limit 100
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF002 -> l_orderkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |  limit: 100
@@ -3190,6 +3296,8 @@ limit 100
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000 -> tpch.lineitem.l_orderkey, RF002 -> l_orderkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |  limit: 100
@@ -3300,6 +3408,8 @@ where
     )
   )
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice * (1 - l_discount))
 |
@@ -3315,6 +3425,8 @@ where
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000 -> l_partkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice * (1 - l_discount))
 |
@@ -3337,6 +3449,8 @@ where
    partitions=1/1 files=1 size=718.94MB
    runtime filters: RF000 -> l_partkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice * (1 - l_discount))
 |
@@ -3403,6 +3517,8 @@ where
 order by
   s_name
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:SORT
 |  order by: s_name ASC
 |
@@ -3448,6 +3564,8 @@ order by
    predicates: l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
    runtime filters: RF000 -> tpch.lineitem.l_suppkey, RF001 -> tpch.lineitem.l_partkey, RF002 -> tpch.lineitem.l_suppkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_name ASC
 |
@@ -3512,6 +3630,8 @@ order by
    predicates: l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
    runtime filters: RF000 -> tpch.lineitem.l_suppkey, RF001 -> tpch.lineitem.l_partkey, RF002 -> tpch.lineitem.l_suppkey
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 18:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_name ASC
 |
@@ -3635,6 +3755,8 @@ order by
   s_name
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:TOP-N [LIMIT=100]
 |  order by: count(*) DESC, s_name ASC
 |
@@ -3688,6 +3810,8 @@ limit 100
    partitions=1/1 files=1 size=718.94MB
    predicates: l3.l_receiptdate > l3.l_commitdate
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 21:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC, s_name ASC
 |  limit: 100
@@ -3763,6 +3887,8 @@ limit 100
    partitions=1/1 files=1 size=718.94MB
    predicates: l3.l_receiptdate > l3.l_commitdate
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 21:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC, s_name ASC
 |  limit: 100
@@ -3895,6 +4021,8 @@ group by
 order by
   cntrycode
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: cntrycode ASC
 |
@@ -3922,6 +4050,8 @@ order by
 03:SCAN HDFS [tpch.orders]
    partitions=1/1 files=1 size=162.56MB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cntrycode ASC
 |
@@ -3969,6 +4099,8 @@ order by
 03:SCAN HDFS [tpch.orders]
    partitions=1/1 files=1 size=162.56MB
 ---- PARALLELPLANS
+PLAN-ROOT SINK
+|
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cntrycode ASC
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
index 6c70c35..0b7f35d 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-kudu.test
@@ -21,6 +21,8 @@ order by
   l_returnflag,
   l_linestatus
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: l_returnflag ASC, l_linestatus ASC
 |
@@ -77,6 +79,8 @@ order by
   p_partkey
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 18:TOP-N [LIMIT=100]
 |  order by: s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC
 |
@@ -155,6 +159,8 @@ order by
   o_orderdate
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=10]
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 2) DESC, o_orderdate ASC
 |
@@ -200,6 +206,8 @@ group by
 order by
   o_orderpriority
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: o_orderpriority ASC
 |
@@ -242,6 +250,8 @@ group by
 order by
   revenue desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 2) DESC
 |
@@ -289,6 +299,8 @@ where
   and l_discount between 0.05 and 0.07
   and l_quantity < 24
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice * l_discount)
 |
@@ -335,6 +347,8 @@ order by
   cust_nation,
   l_year
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: supp_nation ASC, cust_nation ASC, l_year ASC
 |
@@ -410,6 +424,8 @@ group by
 order by
   o_year
 ---- PLAN
+PLAN-ROOT SINK
+|
 16:SORT
 |  order by: o_year ASC
 |
@@ -490,6 +506,8 @@ order by
   nation,
   o_year desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: nation ASC, o_year DESC
 |
@@ -560,6 +578,8 @@ order by
   revenue desc
 limit 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:TOP-N [LIMIT=20]
 |  order by: round(sum(l_extendedprice * (1 - l_discount)), 1) DESC
 |
@@ -621,6 +641,8 @@ where
 order by
   value desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 13:SORT
 |  order by: value DESC
 |
@@ -690,6 +712,8 @@ group by
 order by
   l_shipmode
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: l_shipmode ASC
 |
@@ -728,6 +752,8 @@ order by
   custdist desc,
   c_count desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:SORT
 |  order by: count(*) DESC, c_count DESC
 |
@@ -762,6 +788,8 @@ where
   and l_shipdate >= '1995-09-01'
   and l_shipdate < '1995-10-01'
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0.0 END), sum(l_extendedprice * (1 - l_discount))
 |
@@ -805,6 +833,8 @@ where
 order by
   s_suppkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:SORT
 |  order by: s_suppkey ASC
 |
@@ -865,6 +895,8 @@ order by
   p_type,
   p_size
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
 |
@@ -908,6 +940,8 @@ where
       l_partkey = p_partkey
   )
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice)
 |
@@ -965,6 +999,8 @@ order by
   o_orderdate
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:TOP-N [LIMIT=100]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |
@@ -1031,6 +1067,8 @@ where
     )
   )
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice * (1 - l_discount))
 |
@@ -1080,6 +1118,8 @@ where
 order by
   s_name
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:SORT
 |  order by: s_name ASC
 |
@@ -1155,6 +1195,8 @@ order by
   s_name
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:TOP-N [LIMIT=100]
 |  order by: count(*) DESC, s_name ASC
 |
@@ -1231,6 +1273,8 @@ group by
 order by
   cntrycode
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: cntrycode ASC
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
index caa3420..3ecf764 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-nested.test
@@ -22,6 +22,8 @@ order by
   l_returnflag,
   l_linestatus
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: l_returnflag ASC, l_linestatus ASC
 |
@@ -33,6 +35,8 @@ order by
    partitions=1/1 files=4 size=577.87MB
    predicates: l_shipdate <= '1998-09-02'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 05:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_returnflag ASC, l_linestatus ASC
 |
@@ -96,6 +100,8 @@ order by
   p_partkey
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 26:TOP-N [LIMIT=100]
 |  order by: s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC
 |
@@ -173,6 +179,8 @@ limit 100
    predicates: !empty(s.s_partsupps)
    runtime filters: RF002 -> s_nationkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 33:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC
 |  limit: 100
@@ -292,6 +300,8 @@ order by
   o_orderdate
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=10]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC, o_orderdate ASC
 |
@@ -321,6 +331,8 @@ limit 10
    predicates on o: !empty(o.o_lineitems), o_orderdate < '1995-03-15'
    predicates on l: l_shipdate > '1995-03-15'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC, o_orderdate ASC
 |  limit: 10
@@ -384,6 +396,8 @@ group by
 order by
   o_orderpriority
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:SORT
 |  order by: o_orderpriority ASC
 |
@@ -413,6 +427,8 @@ order by
    predicates on o: o_orderdate >= '1993-07-01', o_orderdate < '1993-10-01'
    predicates on o_lineitems: l_commitdate < l_receiptdate
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_orderpriority ASC
 |
@@ -475,6 +491,8 @@ group by
 order by
   revenue desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 18:SORT
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -528,6 +546,8 @@ order by
    predicates on o: !empty(o.o_lineitems), o_orderdate >= '1994-01-01', o_orderdate < '1995-01-01'
    runtime filters: RF000 -> c.c_nationkey, RF001 -> c_nationkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 23:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -606,6 +626,8 @@ where
   and l_discount between 0.05 and 0.07
   and l_quantity < 24
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice * l_discount)
 |
@@ -613,6 +635,8 @@ where
    partitions=1/1 files=4 size=292.35MB
    predicates: l_discount <= 0.07, l_discount >= 0.05, l_quantity < 24, l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice * l_discount)
 |
@@ -664,6 +688,8 @@ order by
   cust_nation,
   l_year
 ---- PLAN
+PLAN-ROOT SINK
+|
 16:SORT
 |  order by: supp_nation ASC, cust_nation ASC, l_year ASC
 |
@@ -716,6 +742,8 @@ order by
    predicates on l: l_shipdate >= '1995-01-01', l_shipdate <= '1996-12-31'
    runtime filters: RF000 -> c_nationkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 22:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: supp_nation ASC, cust_nation ASC, l_year ASC
 |
@@ -820,6 +848,8 @@ group by
 order by
   o_year
 ---- PLAN
+PLAN-ROOT SINK
+|
 22:SORT
 |  order by: o_year ASC
 |
@@ -886,6 +916,8 @@ order by
    predicates on o: !empty(o.o_lineitems), o_orderdate >= '1995-01-01', o_orderdate <= '1996-12-31'
    runtime filters: RF001 -> c_nationkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 29:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_year ASC
 |
@@ -1001,6 +1033,8 @@ order by
   nation,
   o_year desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 16:SORT
 |  order by: nation ASC, o_year DESC
 |
@@ -1050,6 +1084,8 @@ order by
    partitions=1/1 files=4 size=577.87MB
    predicates: !empty(o.o_lineitems)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 22:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: nation ASC, o_year DESC
 |
@@ -1148,6 +1184,8 @@ order by
   revenue desc
 limit 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:TOP-N [LIMIT=20]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -1185,6 +1223,8 @@ limit 20
    predicates on l: l_returnflag = 'R'
    runtime filters: RF000 -> c_nationkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 16:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |  limit: 20
@@ -1268,6 +1308,8 @@ where
 order by
   value desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 17:SORT
 |  order by: value DESC
 |
@@ -1323,6 +1365,8 @@ order by
    predicates: !empty(s.s_partsupps)
    runtime filters: RF000 -> s_nationkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 25:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: value DESC
 |
@@ -1428,6 +1472,8 @@ group by
 order by
   l_shipmode
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:SORT
 |  order by: l_shipmode ASC
 |
@@ -1448,6 +1494,8 @@ order by
    predicates: !empty(o.o_lineitems)
    predicates on l: l_shipmode IN ('MAIL', 'SHIP'), l_commitdate < l_receiptdate, l_shipdate < l_commitdate, l_receiptdate >= '1994-01-01', l_receiptdate < '1995-01-01'
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: l_shipmode ASC
 |
@@ -1499,6 +1547,8 @@ order by
   custdist desc,
   c_count desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: count(*) DESC, c_count DESC
 |
@@ -1522,6 +1572,8 @@ order by
    partitions=1/1 files=4 size=577.87MB
    predicates on c_orders: (NOT o_comment LIKE '%special%requests%')
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 12:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC, c_count DESC
 |
@@ -1576,6 +1628,8 @@ where
   and l_shipdate >= '1995-09-01'
   and l_shipdate < '1995-10-01'
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0.0 END), sum(l_extendedprice * (1 - l_discount))
 |
@@ -1591,6 +1645,8 @@ where
    predicates: l_shipdate < '1995-10-01', l_shipdate >= '1995-09-01'
    runtime filters: RF000 -> l_partkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum:merge(CASE WHEN p_type LIKE 'PROMO%' THEN l_extendedprice * (1 - l_discount) ELSE 0.0 END), sum:merge(l_extendedprice * (1 - l_discount))
 |
@@ -1646,6 +1702,8 @@ where
 order by
   s_suppkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:SORT
 |  order by: s_suppkey ASC
 |
@@ -1679,6 +1737,8 @@ order by
    predicates: l_shipdate < '1996-04-01', l_shipdate >= '1996-01-01'
    runtime filters: RF000 -> l.l_suppkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 17:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_suppkey ASC
 |
@@ -1763,6 +1823,8 @@ order by
   p_type,
   p_size
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:SORT
 |  order by: count(s_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
 |
@@ -1792,6 +1854,8 @@ order by
    partitions=1/1 files=1 size=111.08MB
    predicates: NOT s_comment LIKE '%Customer%Complaints%', !empty(s.s_partsupps)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 13:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(s_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
 |
@@ -1851,6 +1915,8 @@ where
       l_partkey = p_partkey
   )
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice)
 |
@@ -1879,6 +1945,8 @@ where
    partitions=1/1 files=4 size=577.87MB
    runtime filters: RF000 -> l.l_partkey, RF001 -> l_partkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 12:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice)
 |
@@ -1942,6 +2010,8 @@ order by
   o_orderdate
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:TOP-N [LIMIT=100]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |
@@ -1969,6 +2039,8 @@ limit 100
    partitions=1/1 files=4 size=577.87MB
    predicates: !empty(c.c_orders)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |  limit: 100
@@ -2038,6 +2110,8 @@ where
     )
   )
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(l_extendedprice * (1 - l_discount))
 |
@@ -2053,6 +2127,8 @@ where
    partitions=1/1 files=4 size=577.87MB
    runtime filters: RF000 -> l_partkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum:merge(l_extendedprice * (1 - l_discount))
 |
@@ -2110,6 +2186,8 @@ where
 order by
   s_name
 ---- PLAN
+PLAN-ROOT SINK
+|
 13:SORT
 |  order by: s_name ASC
 |
@@ -2158,6 +2236,8 @@ order by
    predicates: l_shipdate < '1995-01-01', l_shipdate >= '1994-01-01'
    runtime filters: RF000 -> l.l_partkey, RF001 -> l.l_suppkey
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 21:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: s_name ASC
 |
@@ -2267,6 +2347,8 @@ order by
   s_name
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 20:TOP-N [LIMIT=100]
 |  order by: count(*) DESC, s_name ASC
 |
@@ -2326,6 +2408,8 @@ limit 100
    predicates on l1: l1.l_receiptdate > l1.l_commitdate
    predicates on l3: l3.l_receiptdate > l3.l_commitdate
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 25:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: count(*) DESC, s_name ASC
 |  limit: 100
@@ -2434,6 +2518,8 @@ group by
 order by
   cntrycode
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:SORT
 |  order by: cntrycode ASC
 |
@@ -2463,6 +2549,8 @@ order by
    partitions=1/1 files=4 size=577.87MB
    predicates: substr(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 15:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: cntrycode ASC
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
index 96409e2..c4b23f5 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/tpch-views.test
@@ -22,6 +22,8 @@ order by
   l_returnflag,
   l_linestatus
 ---- PLAN
+PLAN-ROOT SINK
+|
 02:SORT
 |  order by: l_returnflag ASC, l_linestatus ASC
 |
@@ -80,6 +82,8 @@ order by
   p_partkey
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 18:TOP-N [LIMIT=100]
 |  order by: s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC
 |
@@ -181,6 +185,8 @@ order by
   o_orderdate
 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=10]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC, o_orderdate ASC
 |
@@ -234,6 +240,8 @@ group by
 order by
   o_orderpriority
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: o_orderpriority ASC
 |
@@ -281,6 +289,8 @@ group by
 order by
   revenue desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -345,6 +355,8 @@ where
   and l_discount between 0.05 and 0.07
   and l_quantity < 24
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  output: sum(tpch.lineitem.l_extendedprice * tpch.lineitem.l_discount)
 |
@@ -393,6 +405,8 @@ order by
   cust_nation,
   l_year
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: supp_nation ASC, cust_nation ASC, l_year ASC
 |
@@ -484,6 +498,8 @@ group by
 order by
   o_year
 ---- PLAN
+PLAN-ROOT SINK
+|
 16:SORT
 |  order by: o_year ASC
 |
@@ -585,6 +601,8 @@ order by
   nation,
   o_year desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:SORT
 |  order by: nation ASC, o_year DESC
 |
@@ -670,6 +688,8 @@ order by
   revenue desc
 limit 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:TOP-N [LIMIT=20]
 |  order by: sum(l_extendedprice * (1 - l_discount)) DESC
 |
@@ -741,6 +761,8 @@ where
 order by
   value desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 13:SORT
 |  order by: value DESC
 |
@@ -825,6 +847,8 @@ group by
 order by
   l_shipmode
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:SORT
 |  order by: l_shipmode ASC
 |
@@ -867,6 +891,8 @@ order by
   custdist desc,
   c_count desc
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:SORT
 |  order by: count(*) DESC, c_count DESC
 |
@@ -906,6 +932,8 @@ where
   and l_shipdate >= '1995-09-01'
   and l_shipdate < '1995-10-01'
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(CASE WHEN tpch.part.p_type LIKE 'PROMO%' THEN tpch.lineitem.l_extendedprice * (1 - tpch.lineitem.l_discount) ELSE 0.0 END), sum(tpch.lineitem.l_extendedprice * (1 - tpch.lineitem.l_discount))
 |
@@ -954,6 +982,8 @@ where
 order by
   s_suppkey
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:SORT
 |  order by: s_suppkey ASC
 |
@@ -1020,6 +1050,8 @@ order by
   p_type,
   p_size
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: count(ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC
 |
@@ -1069,6 +1101,8 @@ where
       l_partkey = p_partkey
   )
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  output: sum(tpch.lineitem.l_extendedprice)
 |
@@ -1134,6 +1168,8 @@ order by
   o_orderdate
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:TOP-N [LIMIT=100]
 |  order by: o_totalprice DESC, o_orderdate ASC
 |
@@ -1210,6 +1246,8 @@ where
     )
   )
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  output: sum(tpch.lineitem.l_extendedprice * (1 - tpch.lineitem.l_discount))
 |
@@ -1265,6 +1303,8 @@ where
 order by
   s_name
 ---- PLAN
+PLAN-ROOT SINK
+|
 10:SORT
 |  order by: s_name ASC
 |
@@ -1353,6 +1393,8 @@ order by
   s_name
 limit 100
 ---- PLAN
+PLAN-ROOT SINK
+|
 12:TOP-N [LIMIT=100]
 |  order by: count(*) DESC, s_name ASC
 |
@@ -1443,6 +1485,8 @@ group by
 order by
   cntrycode
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:SORT
 |  order by: cntrycode ASC
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/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 84053a9..4baf6fe 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/union.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/union.test
@@ -8,6 +8,8 @@ select * from
   on (a.month = b.month)
 where b.month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: month = b.month
 |  runtime filters: RF000 <- b.month
@@ -25,6 +27,8 @@ where b.month = 1
    partitions=2/24 files=2 size=40.32KB
    runtime filters: RF000 -> functional.alltypes.month
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, BROADCAST]
@@ -53,6 +57,8 @@ select * from functional.alltypestiny where year=2009 and month=1
 union all
 select * from functional.alltypestiny where year=2009 and month=2
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--03:SCAN HDFS [functional.alltypestiny]
@@ -71,6 +77,8 @@ NODE 2:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -91,6 +99,8 @@ select * from functional.alltypestiny where year=2009 and month=1 order by int_c
 union all
 select * from functional.alltypestiny where year=2009 and month=2 limit 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--04:SCAN HDFS [functional.alltypestiny]
@@ -114,6 +124,8 @@ NODE 2:
 NODE 4:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--07:EXCHANGE [UNPARTITIONED]
@@ -147,6 +159,8 @@ select * from functional.alltypestiny where year=2009 and month=1
 union distinct
 select * from functional.alltypestiny where year=2009 and month=2
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -168,6 +182,8 @@ NODE 2:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 06:AGGREGATE [FINALIZE]
@@ -198,6 +214,8 @@ select * from functional.alltypestiny where year=2009 and month=1
 union all
 select 1,false,1,1,1,10,1.1,10.1,'01/01/09','1',cast('2009-01-01 00:01:00' as timestamp),2009,1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  constant-operands=2
 |
@@ -212,6 +230,8 @@ NODE 1:
 NODE 2:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -232,6 +252,8 @@ select * from functional.alltypestiny where year=2009 and month=1
 union distinct
 select 1,false,1,1,1,10,1.1,10.1,'01/01/09','1',cast('2009-01-01 00:01:00' as timestamp),2009,1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -249,6 +271,8 @@ NODE 1:
 NODE 2:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |
 05:AGGREGATE [FINALIZE]
@@ -279,6 +303,8 @@ union distinct
 (select * from functional.alltypestiny where year=2009 and month=2)
 order by 3 limit 3
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:TOP-N [LIMIT=3]
 |  order by: tinyint_col ASC
 |
@@ -308,6 +334,8 @@ NODE 3:
 NODE 4:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: tinyint_col ASC
 |  limit: 3
@@ -347,6 +375,8 @@ union all
 (select * from functional.alltypestiny where year=2009 and month=2)
 order by 3,4 limit 3
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:TOP-N [LIMIT=3]
 |  order by: tinyint_col ASC, smallint_col ASC
 |
@@ -378,6 +408,8 @@ NODE 5:
 NODE 6:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: tinyint_col ASC, smallint_col ASC
 |  limit: 3
@@ -419,6 +451,8 @@ union all
 (select * from functional.alltypestiny where year=2009 and month=2)
 order by 3,4 limit 4
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:TOP-N [LIMIT=4]
 |  order by: tinyint_col ASC, smallint_col ASC
 |
@@ -450,6 +484,8 @@ NODE 3:
 NODE 6:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: tinyint_col ASC, smallint_col ASC
 |  limit: 4
@@ -488,6 +524,8 @@ NODE 6:
 union all
 select * from functional.alltypestiny where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--03:SCAN HDFS [functional.alltypestiny]
@@ -506,6 +544,8 @@ NODE 2:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -526,6 +566,8 @@ union all
    union all
    select * from functional.alltypestiny where year=2009 and month=2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--03:SCAN HDFS [functional.alltypestiny]
@@ -544,6 +586,8 @@ NODE 2:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 04:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -564,6 +608,8 @@ NODE 3:
 union distinct
 select * from functional.alltypestiny where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -585,6 +631,8 @@ NODE 2:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 06:AGGREGATE [FINALIZE]
@@ -613,6 +661,8 @@ union distinct
    union distinct
    select * from functional.alltypestiny where year=2009 and month=2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -634,6 +684,8 @@ NODE 2:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 06:AGGREGATE [FINALIZE]
@@ -663,6 +715,8 @@ NODE 3:
 union all
 select * from functional.alltypestiny where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--05:SCAN HDFS [functional.alltypestiny]
@@ -686,6 +740,8 @@ NODE 3:
 NODE 5:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -717,6 +773,8 @@ union all
    union distinct
    select * from functional.alltypestiny where year=2009 and month=2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--05:AGGREGATE [FINALIZE]
@@ -740,6 +798,8 @@ NODE 3:
 NODE 4:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -773,6 +833,8 @@ NODE 4:
 union all
 select * from functional.alltypestiny where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--06:SCAN HDFS [functional.alltypestiny]
@@ -801,6 +863,8 @@ NODE 4:
 NODE 6:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -837,6 +901,8 @@ union all
    union all
    select * from functional.alltypestiny where year=2009 and month=2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--06:AGGREGATE [FINALIZE]
@@ -865,6 +931,8 @@ NODE 4:
 NODE 5:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -902,6 +970,8 @@ union all
    (select * from functional.alltypestiny where year=2009 and month=2)
    limit 10)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--06:UNION
@@ -933,6 +1003,8 @@ NODE 4:
 NODE 7:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 12:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -976,6 +1048,8 @@ NODE 7:
 union all
 select * from functional.alltypestiny where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--05:SCAN HDFS [functional.alltypestiny]
@@ -999,6 +1073,8 @@ NODE 3:
 NODE 5:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -1032,6 +1108,8 @@ union all
    (select * from functional.alltypestiny where year=2009 and month=2)
    order by 3 limit 3)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--05:TOP-N [LIMIT=3]
@@ -1055,6 +1133,8 @@ NODE 3:
 NODE 4:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -1087,6 +1167,8 @@ NODE 4:
 union distinct
 select * from functional.alltypestiny where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -1108,6 +1190,8 @@ NODE 2:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 06:AGGREGATE [FINALIZE]
@@ -1137,6 +1221,8 @@ union distinct
    union all
    select * from functional.alltypestiny where year=2009 and month=2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -1158,6 +1244,8 @@ NODE 2:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 06:AGGREGATE [FINALIZE]
@@ -1189,6 +1277,8 @@ NODE 3:
 union distinct
 select * from functional.alltypestiny where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -1215,6 +1305,8 @@ NODE 3:
 NODE 4:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 07:AGGREGATE [FINALIZE]
@@ -1249,6 +1341,8 @@ union distinct
    union all
    select * from functional.alltypestiny where year=2009 and month=2)
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -1275,6 +1369,8 @@ NODE 3:
 NODE 4:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 07:AGGREGATE [FINALIZE]
@@ -1308,6 +1404,8 @@ NODE 4:
 union distinct
 select * from functional.alltypestiny where year=2009 and month=1
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -1334,6 +1432,8 @@ NODE 3:
 NODE 5:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:EXCHANGE [UNPARTITIONED]
 |
 10:AGGREGATE [FINALIZE]
@@ -1375,6 +1475,8 @@ union distinct
    (select * from functional.alltypestiny where year=2009 and month=2)
    order by 3 limit 3)
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -1401,6 +1503,8 @@ NODE 3:
 NODE 4:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:EXCHANGE [UNPARTITIONED]
 |
 10:AGGREGATE [FINALIZE]
@@ -1444,6 +1548,8 @@ union all
          union all
          select * from functional.alltypestiny where year=2009 and month=3)))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--05:SCAN HDFS [functional.alltypestiny]
@@ -1472,6 +1578,8 @@ NODE 4:
 NODE 5:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=3/090301.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -1502,6 +1610,8 @@ union distinct
          union distinct
          select * from functional.alltypestiny where year=2009 and month=3)))
 ---- PLAN
+PLAN-ROOT SINK
+|
 06:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -1533,6 +1643,8 @@ NODE 4:
 NODE 5:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=3/090301.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 08:AGGREGATE [FINALIZE]
@@ -1572,6 +1684,8 @@ union all
          (select * from functional.alltypestiny where year=2009 and month=3)
          order by 3 limit 3)))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--10:AGGREGATE [FINALIZE]
@@ -1613,6 +1727,8 @@ NODE 6:
 NODE 7:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=3/090301.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 17:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -1673,6 +1789,8 @@ union distinct
          (select * from functional.alltypestiny where year=2009 and month=3)
          order by 3 limit 3)))
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |
@@ -1712,6 +1830,8 @@ NODE 5:
 NODE 6:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=3/090301.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 16:EXCHANGE [UNPARTITIONED]
 |
 15:AGGREGATE [FINALIZE]
@@ -1779,6 +1899,8 @@ union all
    (select * from functional.alltypestiny where year=2009 and month=5)
    order by 3 limit 3)
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:UNION
 |
 |--08:AGGREGATE [FINALIZE]
@@ -1841,6 +1963,8 @@ NODE 13:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=4/090401.txt 0:115
 NODE 14:
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 22:EXCHANGE [UNPARTITIONED]
 |
 09:UNION
@@ -1913,6 +2037,8 @@ union all
 (select * from functional.alltypestiny where year=2009 and month=2)
 order by 3 limit 5
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:TOP-N [LIMIT=5]
 |  order by: tinyint_col ASC
 |
@@ -1936,6 +2062,8 @@ NODE 3:
 NODE 4:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 06:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: tinyint_col ASC
 |  limit: 5
@@ -1965,6 +2093,8 @@ union distinct
 (select * from functional.alltypestiny where year=2009 and month=2)
 order by 3 limit 3
 ---- PLAN
+PLAN-ROOT SINK
+|
 07:TOP-N [LIMIT=3]
 |  order by: tinyint_col ASC
 |
@@ -1994,6 +2124,8 @@ NODE 3:
 NODE 5:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=2/090201.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 12:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: tinyint_col ASC
 |  limit: 3
@@ -2039,6 +2171,8 @@ select x.* from
    from functional.alltypestiny where year=2009 and month=1 group by 1, 2) x
 where x.int_col < 5 and x.bool_col = false
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--04:AGGREGATE [FINALIZE]
@@ -2062,6 +2196,8 @@ NODE 1:
 NODE 3:
   HDFS SPLIT hdfs://localhost:20500/test-warehouse/alltypestiny/year=2009/month=1/090101.txt 0:115
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 09:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -2101,9 +2237,13 @@ select 2, 'b', NULL, 20.f
 union all
 select 3, 'c', NULL, 30.f
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=3
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=3
 ====
@@ -2114,12 +2254,16 @@ select 2, 'b', NULL, 20.0f
 union distinct
 select 3, 'c', NULL, 30.0f
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: 1, 'a', null, f
 |
 00:UNION
    constant-operands=3
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: 1, 'a', null, f
 |
@@ -2135,6 +2279,8 @@ select 3, 'c', NULL, 30.f
 union all
 select int_col, string_col, bool_col, float_col from functional.alltypestiny
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:UNION
 |
 |--02:AGGREGATE [FINALIZE]
@@ -2149,6 +2295,8 @@ select int_col, string_col, bool_col, float_col from functional.alltypestiny
 04:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 03:UNION
@@ -2183,6 +2331,8 @@ select int_col from functional.alltypestiny where year=2009 and month=1
 union all
 select 503
 ---- PLAN
+PLAN-ROOT SINK
+|
 05:UNION
 |  constant-operands=1
 |
@@ -2201,6 +2351,8 @@ select 503
 01:SCAN HDFS [functional.alltypestiny]
    partitions=1/4 files=1 size=115B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 10:EXCHANGE [UNPARTITIONED]
 |
 05:UNION
@@ -2239,9 +2391,13 @@ values(2, 'b', NULL, 20.f)
 union all
 values(3, 'c', NULL, 30.f)
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=3
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=3
 ====
@@ -2252,12 +2408,16 @@ values(2, 'b', NULL, 20.f)
 union distinct
 values(3, 'c', NULL, 30.f)
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: 1, 'a', null, f
 |
 00:UNION
    constant-operands=3
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:AGGREGATE [FINALIZE]
 |  group by: 1, 'a', null, f
 |
@@ -2273,6 +2433,8 @@ values(3, 'c', NULL, 30.f)
 union all
 select int_col, string_col, bool_col, float_col from functional.alltypessmall
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:UNION
 |
 |--02:AGGREGATE [FINALIZE]
@@ -2287,6 +2449,8 @@ select int_col, string_col, bool_col, float_col from functional.alltypessmall
 04:SCAN HDFS [functional.alltypessmall]
    partitions=4/4 files=4 size=6.32KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 07:EXCHANGE [UNPARTITIONED]
 |
 03:UNION
@@ -2329,6 +2493,8 @@ select count(*) from (
       cast('2009-01-01 00:02:00.10' as timestamp),2009,1
 ) x
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:AGGREGATE [FINALIZE]
 |  output: count(*)
 |
@@ -2344,6 +2510,8 @@ select count(*) from (
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:AGGREGATE [FINALIZE]
 |  output: count:merge(*)
 |
@@ -2377,6 +2545,8 @@ select x.int_col, x.bool_col, x.month from
    select * from functional.alltypestiny where year=2009) x
 where x.int_col < 5 and x.bool_col = false and x.month = 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HDFS [functional.alltypestiny]
@@ -2387,6 +2557,8 @@ where x.int_col < 5 and x.bool_col = false and x.month = 1
    partitions=1/4 files=1 size=115B
    predicates: functional.alltypestiny.int_col < 5, functional.alltypestiny.bool_col = FALSE
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -2409,6 +2581,8 @@ select 1 from
    select 1, tinyint_col, 3, bigint_col from functional.alltypessmall) t
 where a + b < 100 and c > 20 and d > 50
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HDFS [functional.alltypes]
@@ -2428,8 +2602,12 @@ select * from
    select 10, 20) t
 where a > b
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
 ====
 # Test union merging only partitioned inputs.
@@ -2445,6 +2623,8 @@ select a.id, a.bigint_col
 union all
 select 1000, 2000
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  constant-operands=1
 |
@@ -2469,6 +2649,8 @@ select 1000, 2000
 01:SCAN HDFS [functional.alltypestiny]
    partitions=4/4 files=4 size=460B
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 11:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -2512,6 +2694,8 @@ select id, bigint_col from functional.alltypessmall order by id limit 10
 union all
 select 1000, 2000
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  constant-operands=1
 |
@@ -2527,6 +2711,8 @@ select 1000, 2000
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  constant-operands=1
 |
@@ -2568,6 +2754,8 @@ select a.id, a.bigint_col
 union all
 select 1000, 2000
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  constant-operands=1
 |
@@ -2604,6 +2792,8 @@ select 1000, 2000
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 20:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -2670,6 +2860,8 @@ select 1000, 2000
 # have explain_level=1
 select * from tpch.lineitem UNION ALL (select * from tpch.lineitem) LIMIT 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |  limit: 1
 |
@@ -2681,6 +2873,8 @@ select * from tpch.lineitem UNION ALL (select * from tpch.lineitem) LIMIT 1
 ====
 select l_orderkey from tpch.lineitem UNION DISTINCT (select l_orderkey from tpch.lineitem) LIMIT 1
 ---- PLAN
+PLAN-ROOT SINK
+|
 03:AGGREGATE [FINALIZE]
 |  group by: l_orderkey
 |  limit: 1
@@ -2705,6 +2899,8 @@ select * from
    (select * from functional.alltypestiny order by id limit 20 offset 10)) v
 where v.id < 10 and v.int_col > 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--08:SELECT
@@ -2744,6 +2940,8 @@ select * from
    (select * from functional.alltypestiny order by id limit 20 offset 10)) v
 where v.id < 10 and v.int_col > 20
 ---- PLAN
+PLAN-ROOT SINK
+|
 09:AGGREGATE [FINALIZE]
 |  group by: id, bool_col, tinyint_col, smallint_col, int_col, bigint_col, float_col, double_col, date_string_col, string_col, timestamp_col, year, month
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/values.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/values.test b/testdata/workloads/functional-planner/queries/PlannerTest/values.test
index 2eec954..a27c793 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/values.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/values.test
@@ -1,19 +1,27 @@
 values(1+1, 2, 5.0, 'a')
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=1
 ====
 values(1+1, 2, 5.0, 'a') order by 1 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=10]
 |  order by: 1 + 1 ASC
 |
 00:UNION
    constant-operands=1
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=10]
 |  order by: 1 + 1 ASC
 |
@@ -22,20 +30,28 @@ values(1+1, 2, 5.0, 'a') order by 1 limit 10
 ====
 values((1+1, 2, 5.0, 'a'), (2, 3, 6.0, 'b'), (3, 4, 7.0, 'c'))
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=3
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 00:UNION
    constant-operands=3
 ====
 values((1+1, 2, 5.0, 'a'), (2, 3, 6.0, 'b'), (3, 4, 7.0, 'c')) order by 1 limit 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=10]
 |  order by: 1 + 1 ASC
 |
 00:UNION
    constant-operands=3
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:TOP-N [LIMIT=10]
 |  order by: 1 + 1 ASC
 |

http://git-wip-us.apache.org/repos/asf/incubator-impala/blob/9f61397f/testdata/workloads/functional-planner/queries/PlannerTest/views.test
----------------------------------------------------------------------
diff --git a/testdata/workloads/functional-planner/queries/PlannerTest/views.test b/testdata/workloads/functional-planner/queries/PlannerTest/views.test
index 50bee61..a6322e6 100644
--- a/testdata/workloads/functional-planner/queries/PlannerTest/views.test
+++ b/testdata/workloads/functional-planner/queries/PlannerTest/views.test
@@ -1,9 +1,13 @@
 # Basic test with a view.
 select int_col, string_col from functional.alltypes_view
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -12,6 +16,8 @@ select int_col, string_col from functional.alltypes_view
 # Basic test with a complex view.
 select * from functional.complex_view
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:TOP-N [LIMIT=100]
 |  order by: b.string_col ASC
 |
@@ -32,6 +38,8 @@ select * from functional.complex_view
    predicates: a.bigint_col < 50
    runtime filters: RF000 -> a.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:MERGING-EXCHANGE [UNPARTITIONED]
 |  order by: b.string_col ASC
 |  limit: 100
@@ -67,9 +75,13 @@ select * from functional.complex_view
 # Basic test with a view on a view
 select int_col, string_col from functional.view_view
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -79,6 +91,8 @@ select int_col, string_col from functional.view_view
 select * from functional.alltypes_view union all
 select * from functional.alltypes_view where id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:UNION
 |
 |--02:SCAN HDFS [functional.alltypes]
@@ -88,6 +102,8 @@ select * from functional.alltypes_view where id < 10
 01:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 03:EXCHANGE [UNPARTITIONED]
 |
 00:UNION
@@ -103,10 +119,14 @@ select * from functional.alltypes_view where id < 10
 select t.id from (select id from functional.alltypes_view) t
 where t.id < 10
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
    predicates: functional.alltypes.id < 10
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -117,6 +137,8 @@ where t.id < 10
 select * from functional.alltypes_view t1, functional.alltypes_view_sub t2,
 functional.complex_view t3 where t1.id = t2.x and t2.x = t3.abc
 ---- PLAN
+PLAN-ROOT SINK
+|
 08:HASH JOIN [INNER JOIN]
 |  hash predicates: int_col = count(a.bigint_col)
 |  runtime filters: RF000 <- count(a.bigint_col)
@@ -155,6 +177,8 @@ functional.complex_view t3 where t1.id = t2.x and t2.x = t3.abc
    predicates: functional.alltypes.id > 1
    runtime filters: RF000 -> functional.alltypes.id, RF001 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 16:EXCHANGE [UNPARTITIONED]
 |
 08:HASH JOIN [INNER JOIN, BROADCAST]
@@ -219,6 +243,8 @@ select * from functional.alltypes_view t1
 inner join functional.alltypes_view t2 on (t1.id = t2.id)
 inner join functional.alltypes_view t3 on (t2.id = t3.id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypes.id = functional.alltypes.id
 |  runtime filters: RF000 <- functional.alltypes.id
@@ -238,6 +264,8 @@ inner join functional.alltypes_view t3 on (t2.id = t3.id)
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> functional.alltypes.id, RF001 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -271,6 +299,8 @@ select * from functional.alltypes_view t1
 inner join functional.alltypes_view t2 using(id)
 inner join functional.alltypes_view t3 using(id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypes.id = functional.alltypes.id
 |  runtime filters: RF000 <- functional.alltypes.id
@@ -290,6 +320,8 @@ inner join functional.alltypes_view t3 using(id)
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> functional.alltypes.id, RF001 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -323,6 +355,8 @@ select * from functional.alltypes_view t1
 left outer join functional.alltypes_view t2 using(id)
 full outer join functional.alltypes_view t3 using(id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [FULL OUTER JOIN]
 |  hash predicates: functional.alltypes.id = functional.alltypes.id
 |
@@ -338,6 +372,8 @@ full outer join functional.alltypes_view t3 using(id)
 00:SCAN HDFS [functional.alltypes]
    partitions=24/24 files=24 size=478.45KB
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [FULL OUTER JOIN, PARTITIONED]
@@ -369,6 +405,8 @@ select * from functional.alltypes_view t1
 inner join [broadcast] functional.alltypes_view t2 using(id)
 inner join [shuffle] functional.alltypes_view t3 using(id)
 ---- PLAN
+PLAN-ROOT SINK
+|
 04:HASH JOIN [INNER JOIN]
 |  hash predicates: functional.alltypes.id = functional.alltypes.id
 |  runtime filters: RF000 <- functional.alltypes.id
@@ -388,6 +426,8 @@ inner join [shuffle] functional.alltypes_view t3 using(id)
    partitions=24/24 files=24 size=478.45KB
    runtime filters: RF000 -> functional.alltypes.id, RF001 -> functional.alltypes.id
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 08:EXCHANGE [UNPARTITIONED]
 |
 04:HASH JOIN [INNER JOIN, PARTITIONED]
@@ -419,10 +459,14 @@ inner join [shuffle] functional.alltypes_view t3 using(id)
 # enabling proper partition pruning for this particular view.
 select * from functional.alltypes_parens
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes]
    partitions=1/24 files=1 size=19.95KB
    predicates: (int_col < 100 OR bool_col = FALSE)
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes]
@@ -433,10 +477,14 @@ select * from functional.alltypes_parens
 # possible (see IMPALA-923)
 select bool_col FROM ( SELECT bool_col FROM functional.alltypes t ) t WHERE t.bool_col
 ---- PLAN
+PLAN-ROOT SINK
+|
 00:SCAN HDFS [functional.alltypes t]
    partitions=24/24 files=24 size=478.45KB
    predicates: bool_col
 ---- DISTRIBUTEDPLAN
+PLAN-ROOT SINK
+|
 01:EXCHANGE [UNPARTITIONED]
 |
 00:SCAN HDFS [functional.alltypes t]