You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2019/01/04 00:24:34 UTC

[35/35] hive git commit: HIVE-16957: Support CTAS for auto gather column stats (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)

HIVE-16957: Support CTAS for auto gather column stats (Jesus Camacho Rodriguez, reviewed by Ashutosh Chauhan)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/138b00ca
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/138b00ca
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/138b00ca

Branch: refs/heads/master
Commit: 138b00cad8433780fb605e280121b0a9e8baeac5
Parents: 691c4cb
Author: Jesus Camacho Rodriguez <jc...@apache.org>
Authored: Tue Dec 11 17:24:21 2018 -0800
Committer: Jesus Camacho Rodriguez <jc...@apache.org>
Committed: Fri Jan 4 01:22:41 2019 +0100

----------------------------------------------------------------------
 .../listener/TestDbNotificationListener.java    |    6 +-
 .../apache/hadoop/hive/hooks/TestHs2Hooks.java  |    1 +
 .../ql/optimizer/LimitPushdownOptimizer.java    |   28 +-
 .../hive/ql/parse/BaseSemanticAnalyzer.java     |    2 +-
 .../ql/parse/ColumnStatsAutoGatherContext.java  |  127 +-
 .../ql/parse/ColumnStatsSemanticAnalyzer.java   |  121 +-
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java  |   61 +-
 .../hadoop/hive/ql/plan/CreateViewDesc.java     |    7 +
 .../cbo_rp_cross_product_check_2.q              |    6 +-
 ql/src/test/queries/clientpositive/llap_acid2.q |   17 +-
 .../materialized_view_create_rewrite.q          |    2 -
 .../materialized_view_create_rewrite_2.q        |    4 -
 .../materialized_view_create_rewrite_3.q        |    8 -
 .../materialized_view_create_rewrite_4.q        |   14 +-
 .../materialized_view_create_rewrite_5.q        |    7 -
 .../materialized_view_create_rewrite_dummy.q    |    2 -
 .../materialized_view_create_rewrite_multi_db.q |    2 -
 ...rialized_view_create_rewrite_rebuild_dummy.q |    8 -
 ...terialized_view_create_rewrite_time_window.q |    6 -
 ...rialized_view_create_rewrite_time_window_2.q |    3 -
 .../materialized_view_rewrite_1.q               |   14 -
 .../materialized_view_rewrite_10.q              |    6 -
 .../materialized_view_rewrite_2.q               |   10 -
 .../materialized_view_rewrite_3.q               |    8 -
 .../materialized_view_rewrite_4.q               |   11 -
 .../materialized_view_rewrite_5.q               |   17 -
 .../materialized_view_rewrite_6.q               |    9 -
 .../materialized_view_rewrite_7.q               |    7 -
 .../materialized_view_rewrite_8.q               |    2 -
 .../materialized_view_rewrite_9.q               |    3 -
 .../materialized_view_rewrite_empty.q           |    1 -
 .../materialized_view_rewrite_no_join_opt.q     |    8 -
 .../materialized_view_rewrite_no_join_opt_2.q   |   11 -
 .../materialized_view_rewrite_part_1.q          |   11 -
 .../materialized_view_rewrite_part_2.q          |   10 -
 .../materialized_view_rewrite_ssb.q             |    6 -
 .../materialized_view_rewrite_ssb_2.q           |    6 -
 .../results/clientnegative/masking_mv.q.out     |   31 +-
 .../alter_table_update_status.q.out             |    8 +-
 ..._table_update_status_disable_bitvector.q.out |    4 +-
 .../clientpositive/autoColumnStats_4.q.out      |    1 +
 .../materialized_view_create_rewrite.q.out      |   14 +-
 .../cbo_rp_cross_product_check_2.q.out          |   44 +-
 ql/src/test/results/clientpositive/ctas.q.out   |  255 +-
 .../results/clientpositive/ctas_colname.q.out   |  339 ++-
 .../ctas_uses_database_location.q.out           |   33 +-
 .../results/clientpositive/decimal_serde.q.out  |    4 +-
 .../clientpositive/druid/druidmini_mv.q.out     |   64 +-
 .../results/clientpositive/explain_ddl.q.out    |  124 +-
 .../clientpositive/groupby_duplicate_key.q.out  |   49 +-
 .../results/clientpositive/input1_limit.q.out   |    2 +
 .../results/clientpositive/input3_limit.q.out   |    1 +
 .../results/clientpositive/input_part10.q.out   |    1 +
 .../insert2_overwrite_partitions.q.out          |    2 +
 .../results/clientpositive/insert_into1.q.out   |    3 +
 .../results/clientpositive/insert_into2.q.out   |    3 +
 .../results/clientpositive/insert_into3.q.out   |    4 +
 .../results/clientpositive/insert_into4.q.out   |    2 +
 .../results/clientpositive/insert_into5.q.out   |    1 +
 .../results/clientpositive/insert_into6.q.out   |    1 +
 ql/src/test/results/clientpositive/join42.q.out |  233 +-
 .../clientpositive/join_filters_overlap.q.out   |   20 +-
 .../limit_pushdown_negative.q.out               |    1 +
 .../llap/convert_decimal64_to_decimal.q.out     |   48 +-
 .../clientpositive/llap/cross_prod_1.q.out      |  206 +-
 .../clientpositive/llap/cross_prod_3.q.out      |   14 +-
 .../clientpositive/llap/cross_prod_4.q.out      |   16 +-
 .../llap/cross_product_check_1.q.out            |  150 +-
 .../llap/cross_product_check_2.q.out            |  144 +-
 .../test/results/clientpositive/llap/ctas.q.out |  195 +-
 .../test/results/clientpositive/llap/dpp.q.out  |  100 +-
 .../llap/dynamic_partition_pruning.q.out        | 1019 +++----
 .../llap/dynamic_semijoin_reduction.q.out       |   24 +-
 .../llap/dynamic_semijoin_reduction_3.q.out     |   48 -
 .../llap/dynpart_sort_opt_vectorization.q.out   |    4 +
 .../llap/dynpart_sort_optimization.q.out        |    5 +
 .../clientpositive/llap/empty_join.q.out        |    4 +-
 .../llap/enforce_constraint_notnull.q.out       |  122 +-
 .../clientpositive/llap/explainuser_1.q.out     |  118 +-
 .../llap/insert1_overwrite_partitions.q.out     |    3 +
 .../clientpositive/llap/insert_into1.q.out      |    3 +
 .../clientpositive/llap/insert_into2.q.out      |    3 +
 .../llap/insert_into_default_keyword.q.out      |   24 -
 .../results/clientpositive/llap/lineage2.q.out  |   10 +-
 .../results/clientpositive/llap/lineage3.q.out  |    2 +-
 .../clientpositive/llap/llap_acid2.q.out        |  570 ++--
 .../clientpositive/llap/llap_nullscan.q.out     |   78 +-
 .../clientpositive/llap/llap_partitioned.q.out  |   12 +-
 .../results/clientpositive/llap/llap_udf.q.out  |   39 +-
 .../clientpositive/llap/llapdecider.q.out       |   39 +-
 .../llap/materialized_view_create.q.out         |    6 +-
 .../llap/materialized_view_create_rewrite.q.out |   24 +-
 .../materialized_view_create_rewrite_2.q.out    |   62 +-
 .../materialized_view_create_rewrite_3.q.out    |  109 +-
 .../materialized_view_create_rewrite_4.q.out    |  232 +-
 .../materialized_view_create_rewrite_5.q.out    |   50 +-
 ...materialized_view_create_rewrite_dummy.q.out |   24 +-
 ...erialized_view_create_rewrite_multi_db.q.out |   14 +-
 ...ized_view_create_rewrite_rebuild_dummy.q.out |  111 +-
 ...alized_view_create_rewrite_time_window.q.out |  105 +-
 ...ized_view_create_rewrite_time_window_2.q.out |   34 +-
 .../llap/materialized_view_describe.q.out       |   10 +-
 .../llap/materialized_view_partitioned.q.out    |  107 +-
 .../llap/materialized_view_partitioned_3.q.out  |   51 +-
 .../llap/materialized_view_rewrite_1.q.out      |  140 -
 .../llap/materialized_view_rewrite_10.q.out     |   60 -
 .../llap/materialized_view_rewrite_2.q.out      |  102 +-
 .../llap/materialized_view_rewrite_3.q.out      |   80 -
 .../llap/materialized_view_rewrite_4.q.out      |  110 -
 .../llap/materialized_view_rewrite_5.q.out      |  178 +-
 .../llap/materialized_view_rewrite_6.q.out      |   90 -
 .../llap/materialized_view_rewrite_7.q.out      |   70 -
 .../llap/materialized_view_rewrite_8.q.out      |   20 -
 .../llap/materialized_view_rewrite_9.q.out      |   30 -
 .../llap/materialized_view_rewrite_empty.q.out  |   10 -
 .../materialized_view_rewrite_no_join_opt.q.out |   80 -
 ...aterialized_view_rewrite_no_join_opt_2.q.out |  186 --
 .../llap/materialized_view_rewrite_part_1.q.out |  186 --
 .../llap/materialized_view_rewrite_part_2.q.out |  138 +-
 .../llap/materialized_view_rewrite_ssb.q.out    |  284 +-
 .../llap/materialized_view_rewrite_ssb_2.q.out  |  286 +-
 .../llap/multi_insert_lateral_view.q.out        |  440 +--
 .../llap/orc_llap_nonvector.q.out               |   16 +-
 .../clientpositive/llap/partition_ctas.q.out    |   45 +-
 .../clientpositive/llap/rcfile_createas1.q.out  |   39 +-
 .../clientpositive/llap/results_cache_1.q.out   |   26 +-
 .../llap/results_cache_invalidation2.q.out      |   32 +-
 .../llap/results_cache_with_masking.q.out       |   24 +-
 .../llap/runtime_stats_merge.q.out              |   32 +-
 .../results/clientpositive/llap/semijoin.q.out  |  524 ++--
 .../clientpositive/llap/semijoin_hint.q.out     |   48 -
 .../results/clientpositive/llap/sqlmerge.q.out  |   24 -
 .../clientpositive/llap/sqlmerge_stats.q.out    |  168 --
 .../clientpositive/llap/subquery_scalar.q.out   |    2 +
 .../clientpositive/llap/subquery_select.q.out   |   26 +-
 .../results/clientpositive/llap/tez_dml.q.out   |   73 +-
 .../clientpositive/llap/tez_nway_join.q.out     |  122 +-
 .../llap/tez_smb_reduce_side.q.out              |   30 +-
 .../clientpositive/llap/unionDistinct_1.q.out   | 1048 +++----
 .../clientpositive/llap/union_top_level.q.out   |   85 +-
 .../llap/vector_between_columns.q.out           |   38 +-
 .../clientpositive/llap/vector_between_in.q.out |  166 +-
 .../llap/vector_case_when_1.q.out               |   18 +-
 .../llap/vector_char_mapjoin1.q.out             |  214 +-
 .../llap/vector_char_varchar_1.q.out            |  136 +-
 .../clientpositive/llap/vector_count.q.out      |   48 +-
 .../llap/vector_decimal_10_0.q.out              |   10 +-
 .../clientpositive/llap/vector_decimal_6.q.out  |   47 +-
 .../llap/vector_decimal_aggregate.q.out         |   64 +-
 .../llap/vector_decimal_cast.q.out              |   10 +-
 .../llap/vector_decimal_math_funcs.q.out        |   16 +-
 .../clientpositive/llap/vector_groupby4.q.out   |   26 +-
 .../clientpositive/llap/vector_groupby6.q.out   |   26 +-
 .../llap/vector_groupby_grouping_id1.q.out      |   84 +-
 .../llap/vector_groupby_grouping_id2.q.out      |  212 +-
 .../llap/vector_groupby_grouping_id3.q.out      |   32 +-
 .../llap/vector_groupby_grouping_sets1.q.out    |   94 +-
 .../llap/vector_groupby_grouping_sets2.q.out    |   54 +-
 .../llap/vector_groupby_grouping_sets3.q.out    |   46 +-
 .../vector_groupby_grouping_sets3_dec.q.out     |   46 +-
 .../llap/vector_groupby_grouping_sets4.q.out    |   94 +-
 .../llap/vector_groupby_grouping_sets5.q.out    |   64 +-
 .../llap/vector_groupby_grouping_sets6.q.out    |   24 +-
 .../vector_groupby_grouping_sets_grouping.q.out |  222 +-
 .../vector_groupby_grouping_sets_limit.q.out    |  128 +-
 .../llap/vector_grouping_sets.q.out             |   40 +-
 .../clientpositive/llap/vector_if_expr_2.q.out  |   10 +-
 .../llap/vector_include_no_sel.q.out            |   22 +-
 .../llap/vector_interval_mapjoin.q.out          |   20 +-
 .../clientpositive/llap/vector_join30.q.out     |  398 +--
 .../llap/vector_join_filters.q.out              |  120 +-
 .../clientpositive/llap/vector_join_nulls.q.out |  110 +-
 .../llap/vector_leftsemi_mapjoin.q.out          | 2724 +++++++++---------
 .../clientpositive/llap/vector_like_2.q.out     |   10 +-
 .../llap/vector_llap_io_data_conversion.q.out   |   10 +-
 .../llap/vector_nullsafe_join.q.out             |  200 +-
 .../llap/vector_number_compare_projection.q.out |   24 +-
 .../llap/vector_partitioned_date_time.q.out     |   60 +-
 .../llap/vector_reduce_groupby_decimal.q.out    |   20 +-
 .../clientpositive/llap/vector_udf2.q.out       |   16 +-
 .../llap/vector_varchar_mapjoin1.q.out          |  162 +-
 .../llap/vector_windowing_navfn.q.out           |   48 +-
 .../llap/vector_windowing_streaming.q.out       |   82 +-
 .../llap/vectorization_decimal_date.q.out       |   10 +-
 .../llap/vectorization_short_regress.q.out      |   60 +-
 .../vectorized_dynamic_partition_pruning.q.out  |  993 ++++---
 .../vectorized_dynamic_semijoin_reduction.q.out |  266 +-
 .../llap/vectorized_mapjoin3.q.out              |   72 +-
 .../clientpositive/load_dyn_part14.q.out        |    3 +
 .../results/clientpositive/masking_mv.q.out     |   66 +-
 ql/src/test/results/clientpositive/merge3.q.out |   57 +-
 ql/src/test/results/clientpositive/merge4.q.out |    2 +
 .../results/clientpositive/mm_default.q.out     |    4 +-
 .../clientpositive/named_column_join.q.out      |    2 +-
 .../results/clientpositive/nonmr_fetch.q.out    |   49 +-
 .../nonreserved_keywords_insert_into1.q.out     |    3 +
 .../results/clientpositive/nullformatCTAS.q.out |   31 +-
 .../results/clientpositive/orc_createas1.q.out  |   60 +
 .../clientpositive/parallel_orderby.q.out       |   29 +-
 .../clientpositive/perf/spark/query70.q.out     |    1 +
 .../perf/tez/constraints/query70.q.out          |  106 +-
 .../clientpositive/perf/tez/query70.q.out       |  142 +-
 .../clientpositive/rcfile_default_format.q.out  |    8 +-
 .../sample_islocalmode_hook_use_metadata.q.out  |    2 +-
 .../show_materialized_views.q.out               |    8 +-
 .../clientpositive/skewjoin_noskew.q.out        |   63 +-
 .../clientpositive/skewjoin_onesideskew.q.out   |   78 +-
 .../results/clientpositive/smb_mapjoin9.q.out   |   25 +
 .../clientpositive/spark/subquery_scalar.q.out  |    2 +
 .../clientpositive/spark/subquery_select.q.out  |   17 +-
 ql/src/test/results/clientpositive/stats5.q.out |    2 +-
 .../clientpositive/tez/explainanalyze_1.q.out   |   31 +-
 .../clientpositive/tez/explainanalyze_3.q.out   |   39 +-
 .../clientpositive/tez/explainanalyze_5.q.out   |    8 +-
 .../clientpositive/tez/explainuser_3.q.out      |   33 +-
 .../results/clientpositive/udtf_explode.q.out   |    4 +
 .../test/results/clientpositive/union24.q.out   |   48 +-
 .../test/results/clientpositive/union25.q.out   |   55 +-
 .../clientpositive/vector_decimal_6.q.out       |   24 +
 219 files changed, 9418 insertions(+), 9589 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
----------------------------------------------------------------------
diff --git a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
index be4f9ae..9eacfd8 100644
--- a/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
+++ b/itests/hcatalog-unit/src/test/java/org/apache/hive/hcatalog/listener/TestDbNotificationListener.java
@@ -1280,12 +1280,12 @@ public class TestDbNotificationListener {
     driver.run("create table " + sourceTblName + " (c int)");
     // Event 2 (alter: marker stats event), 3 (insert), 4 (alter: stats update event)
     driver.run("insert into table " + sourceTblName + " values (1)");
-    // Event 5, 6 (alter: stats update event)
+    // Event 5, 6 (alter), 7 (alter: stats update event)
     driver.run("create table " + targetTblName + " as select c from " + sourceTblName);
 
     // Get notifications from metastore
     NotificationEventResponse rsp = msClient.getNextNotification(firstEventId, 0, null);
-    assertEquals(7, rsp.getEventsSize());
+    assertEquals(8, rsp.getEventsSize());
     NotificationEvent event = rsp.getEvents().get(0);
     assertEquals(firstEventId + 1, event.getEventId());
     assertEquals(EventType.CREATE_TABLE.toString(), event.getEventType());
@@ -1299,7 +1299,7 @@ public class TestDbNotificationListener {
     event = rsp.getEvents().get(5);
     assertEquals(firstEventId + 6, event.getEventId());
     assertEquals(EventType.CREATE_TABLE.toString(), event.getEventType());
-    testEventCounts(defaultDbName, firstEventId, null, null, 7);
+    testEventCounts(defaultDbName, firstEventId, null, null, 8);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java
index d26af3b..509b178 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/hooks/TestHs2Hooks.java
@@ -147,6 +147,7 @@ public class TestHs2Hooks {
     hiveConf.setVar(ConfVars.SEMANTIC_ANALYZER_HOOK,
         SemanticAnalysisHook.class.getName());
     hiveConf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    hiveConf.setBoolVar(ConfVars.HIVESTATSCOLAUTOGATHER, false);
 
     hiveServer2 = new HiveServer2();
     hiveServer2.init(hiveConf);

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java
index 859c18f..1dbe160 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/LimitPushdownOptimizer.java
@@ -133,9 +133,17 @@ public class LimitPushdownOptimizer extends Transform {
         }
       }
       if (rs != null) {
-        if (OperatorUtils.findOperators(rs, GroupByOperator.class).size() > 1){
-          // Not safe to continue for RS-GBY-GBY-LIM kind of pipelines. See HIVE-10607 for more.
-          return false;
+        Operator<?> currentOp = rs;
+        boolean foundGroupByOperator = false;
+        while (currentOp != nd) { // nd = limitOp
+          if (currentOp instanceof GroupByOperator) {
+            if (foundGroupByOperator) {
+              // Not safe to continue for RS-GBY-GBY-LIM kind of pipelines. See HIVE-10607 for more.
+              return false;
+            }
+            foundGroupByOperator = true;
+          }
+          currentOp = currentOp.getChildOperators().get(0);
         }
         LimitOperator limit = (LimitOperator) nd;
         LimitDesc limitDesc = limit.getConf();
@@ -175,9 +183,17 @@ public class LimitPushdownOptimizer extends Transform {
         }
       }
       if (pRS != null) {
-        if (OperatorUtils.findOperators(pRS, GroupByOperator.class).size() > 1){
-          // Not safe to continue for RS-GBY-GBY-LIM kind of pipelines. See HIVE-10607 for more.
-          return false;
+        Operator<?> currentOp = pRS;
+        boolean foundGroupByOperator = false;
+        while (currentOp != nd) { // nd = cRS
+          if (currentOp instanceof GroupByOperator) {
+            if (foundGroupByOperator) {
+              // Not safe to continue for RS-GBY-GBY-LIM kind of pipelines. See HIVE-10607 for more.
+              return false;
+            }
+            foundGroupByOperator = true;
+          }
+          currentOp = currentOp.getChildOperators().get(0);
         }
         List<ExprNodeDesc> cKeys = cRS.getConf().getKeyCols();
         List<ExprNodeDesc> pKeys = pRS.getConf().getKeyCols();

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
index 3a51d97..833757c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/BaseSemanticAnalyzer.java
@@ -1636,7 +1636,7 @@ public abstract class BaseSemanticAnalyzer {
     }
   }
 
-  public class AnalyzeRewriteContext {
+  public static class AnalyzeRewriteContext {
 
     private String tableName;
     private List<String> colName;

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
index 11ccff4..acd2b6d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsAutoGatherContext.java
@@ -25,6 +25,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.QueryState;
@@ -32,6 +33,8 @@ import org.apache.hadoop.hive.ql.exec.ColumnInfo;
 import org.apache.hadoop.hive.ql.exec.Operator;
 import org.apache.hadoop.hive.ql.exec.RowSchema;
 import org.apache.hadoop.hive.ql.exec.SelectOperator;
+import org.apache.hadoop.hive.ql.exec.UDTFOperator;
+import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.AnalyzeRewriteContext;
@@ -47,10 +50,9 @@ import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
 
 /**
  * ColumnStatsAutoGatherContext: This is passed to the compiler when set
- * hive.stats.autogather=true during the INSERT OVERWRITE command.
- *
- **/
-
+ * hive.stats.autogather=true during the INSERT, INSERT OVERWRITE, or CTAS
+ * commands.
+ */
 public class ColumnStatsAutoGatherContext {
 
   public AnalyzeRewriteContext analyzeRewrite;
@@ -92,79 +94,134 @@ public class ColumnStatsAutoGatherContext {
     this.analyzeRewrite = analyzeRewrite;
   }
 
+  /**
+   * Generate the statement of analyze table [tablename] compute statistics for columns
+   * In non-partitioned table case, it will generate TS-SEL-GBY-RS-GBY-SEL-FS operator
+   * In static-partitioned table case, it will generate TS-FIL(partitionKey)-SEL-GBY(partitionKey)-RS-GBY-SEL-FS operator
+   * In dynamic-partitioned table case, it will generate TS-SEL-GBY(partitionKey)-RS-GBY-SEL-FS operator
+   * However, we do not need to specify the partition-spec because (1) the data is going to be inserted to that specific partition
+   * (2) we can compose the static/dynamic partition using a select operator in replaceSelectOperatorProcess.
+   */
   public void insertAnalyzePipeline() throws SemanticException{
-    // 1. Generate the statement of analyze table [tablename] compute statistics for columns
-    // In non-partitioned table case, it will generate TS-SEL-GBY-RS-GBY-SEL-FS operator
-    // In static-partitioned table case, it will generate TS-FIL(partitionKey)-SEL-GBY(partitionKey)-RS-GBY-SEL-FS operator
-    // In dynamic-partitioned table case, it will generate TS-SEL-GBY(partitionKey)-RS-GBY-SEL-FS operator
-    // However, we do not need to specify the partition-spec because (1) the data is going to be inserted to that specific partition
-    // (2) we can compose the static/dynamic partition using a select operator in replaceSelectOperatorProcess..
     String analyzeCommand = "analyze table `" + tbl.getDbName() + "`.`" + tbl.getTableName() + "`"
         + " compute statistics for columns ";
+    insertAnalyzePipeline(analyzeCommand, false);
+  }
 
-    // 2. Based on the statement, generate the selectOperator
+  /**
+   * Generate the statement of SELECT compute_stats(col1) compute_stats(col2),...,
+   * similar to the one generated from ANALYZE TABLE t1 COMPUTE STATISTICS FOR COLUMNS,
+   * but t1 is replaced by a TABLE(VALUES(cast(null as int),cast(null as string))) AS t1(col1,col2).
+   *
+   * We use TABLE-VALUES statement for computing stats for CTAS statement because in those cases
+   * the table has not been created yet. Once the plan for the SELECT statement is generated,
+   * we connect it to the existing CTAS plan as we do for INSERT or INSERT OVERWRITE.
+   */
+  public void insertTableValuesAnalyzePipeline() throws SemanticException {
+    // Instead of starting from analyze statement, we just generate the Select plan
+    boolean isPartitionStats = conf.getBoolVar(ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS) && tbl.isPartitioned();
+    if (isPartitionStats) {
+      partSpec = new HashMap<>();
+      List<String> partKeys = Utilities.getColumnNamesFromFieldSchema(tbl.getPartitionKeys());
+      for (String partKey : partKeys) {
+        partSpec.put(partKey, null);
+      }
+    }
+    String command = ColumnStatsSemanticAnalyzer.genRewrittenQuery(
+        tbl, Utilities.getColumnNamesFromFieldSchema(tbl.getCols()), conf, partSpec, isPartitionStats, true);
+    insertAnalyzePipeline(command, true);
+  }
+
+  private void insertAnalyzePipeline(String command, boolean rewritten) throws SemanticException {
+    // 1. Based on the statement, generate the selectOperator
     Operator<?> selOp = null;
     try {
-      selOp = genSelOpForAnalyze(analyzeCommand, origCtx);
+      selOp = genSelOp(command, rewritten, origCtx);
     } catch (IOException | ParseException e) {
       throw new SemanticException(e);
     }
 
-    // 3. attach this SEL to the operator right before FS
+    // 2. attach this SEL to the operator right before FS
     op.getChildOperators().add(selOp);
     selOp.getParentOperators().clear();
     selOp.getParentOperators().add(op);
 
-    // 4. address the colExp, colList, etc for the SEL
+    // 3. address the colExp, colList, etc for the SEL
     try {
       replaceSelectOperatorProcess((SelectOperator)selOp, op);
     } catch (HiveException e) {
       throw new SemanticException(e);
     }
   }
-  
-  @SuppressWarnings("rawtypes")
-  private Operator genSelOpForAnalyze(String analyzeCommand, Context origCtx) throws IOException, ParseException, SemanticException{
-    //0. initialization
+
+  private Operator genSelOp(String command, boolean rewritten, Context origCtx)
+      throws IOException, ParseException, SemanticException {
+    // 1. initialization
     Context ctx = new Context(conf);
     ctx.setOpContext(origCtx.getOpContext());
     ctx.setExplainConfig(origCtx.getExplainConfig());
-    ASTNode tree = ParseUtils.parse(analyzeCommand, ctx);
 
-    //1. get the ColumnStatsSemanticAnalyzer
-    QueryState queryState = new QueryState.Builder().withHiveConf(conf).build();
-    BaseSemanticAnalyzer baseSem = SemanticAnalyzerFactory.get(queryState, tree);
-    ColumnStatsSemanticAnalyzer colSem = (ColumnStatsSemanticAnalyzer) baseSem;
+    // 2. parse tree and create semantic analyzer. if we need to rewrite the analyze
+    // statement, we do it now
+    final ASTNode ast;
+    final SemanticAnalyzer sem;
+    final QueryState queryState = new QueryState.Builder().withHiveConf(conf).build();
+    if (rewritten) {
+      // Create the context object that is needed to store the column stats
+      this.analyzeRewrite = ColumnStatsSemanticAnalyzer.genAnalyzeRewriteContext(conf, tbl);
+
+      // The analyze statement has already been rewritten, we just need to create the AST
+      // and the corresponding semantic analyzer
+      ast = ParseUtils.parse(command, ctx);
+      BaseSemanticAnalyzer baseSem = SemanticAnalyzerFactory.get(queryState, ast);
+      sem = (SemanticAnalyzer) baseSem;
+    } else {
+      // We need to rewrite the analyze command and get the rewritten AST
+      ASTNode analyzeTree = ParseUtils.parse(command, ctx);
+      BaseSemanticAnalyzer baseSem = SemanticAnalyzerFactory.get(queryState, analyzeTree);
+      ColumnStatsSemanticAnalyzer colSem = (ColumnStatsSemanticAnalyzer) baseSem;
+      ast = colSem.rewriteAST(analyzeTree, this);
 
-    //2. get the rewritten AST
-    ASTNode ast = colSem.rewriteAST(tree, this);
-    baseSem = SemanticAnalyzerFactory.get(queryState, ast);
-    SemanticAnalyzer sem = (SemanticAnalyzer) baseSem;
+      // Obtain the context object that is needed to store the column stats
+      this.analyzeRewrite = colSem.getAnalyzeRewriteContext();
+
+      // Analyze the rewritten statement
+      baseSem = SemanticAnalyzerFactory.get(queryState, ast);
+      sem = (SemanticAnalyzer) baseSem;
+    }
     QB qb = new QB(null, null, false);
     ASTNode child = ast;
-    ParseContext subPCtx = ((SemanticAnalyzer) sem).getParseContext();
+    ParseContext subPCtx = sem.getParseContext();
     subPCtx.setContext(ctx);
-    ((SemanticAnalyzer) sem).initParseCtx(subPCtx);
+    sem.initParseCtx(subPCtx);
     sem.doPhase1(child, qb, sem.initPhase1Ctx(), null);
     // This will trigger new calls to metastore to collect metadata
     // TODO: cache the information from the metastore
     sem.getMetaData(qb);
-    Operator<?> operator = sem.genPlan(qb);
+    sem.genPlan(qb);
 
-    //3. populate the load file work so that ColumnStatsTask can work
+    // 3. populate the load file work so that ColumnStatsTask can work
     loadFileWork.addAll(sem.getLoadFileWork());
 
-    //4. because there is only one TS for analyze statement, we can get it.
+    // 4. because there is only one TS for analyze statement, we can get it.
     if (sem.topOps.values().size() != 1) {
       throw new SemanticException(
           "ColumnStatsAutoGatherContext is expecting exactly one TS, but finds "
               + sem.topOps.values().size());
     }
-    operator = sem.topOps.values().iterator().next();
+    Operator<?> operator = sem.topOps.values().iterator().next();
 
-    //5. get the first SEL after TS
-    while(!(operator instanceof SelectOperator)){
+    // 5. if this has been rewritten, get the SEL after UDTF;
+    // otherwise, get the first SEL after TS
+    if (rewritten) {
+      while (!(operator instanceof UDTFOperator)) {
+        operator = operator.getChildOperators().get(0);
+      }
       operator = operator.getChildOperators().get(0);
+    } else {
+      while (!(operator instanceof SelectOperator)) {
+        operator = operator.getChildOperators().get(0);
+      }
     }
     return operator;
   }

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
index 9aff006..066807b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
@@ -148,7 +148,8 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     }
   }
 
-  private StringBuilder genPartitionClause(Map<String, String> partSpec) throws SemanticException {
+  private static StringBuilder genPartitionClause(Table tbl, Map<String, String> partSpec)
+      throws SemanticException {
     StringBuilder whereClause = new StringBuilder(" where ");
     boolean predPresent = false;
     StringBuilder groupByClause = new StringBuilder(" group by ");
@@ -162,7 +163,8 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
         } else {
           whereClause.append(" and ");
         }
-        whereClause.append("`").append(partKey).append("` = ").append(genPartValueString(getColTypeOf(partKey), value));
+        whereClause.append("`").append(partKey).append("` = ")
+            .append(genPartValueString(getColTypeOf(tbl, partKey), value));
       }
     }
 
@@ -181,8 +183,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
 
 
 
-  private String getColTypeOf(String partKey) throws SemanticException{
-
+  private static String getColTypeOf(Table tbl, String partKey) throws SemanticException{
     for (FieldSchema fs : tbl.getPartitionKeys()) {
       if (partKey.equalsIgnoreCase(fs.getName())) {
         return fs.getType().toLowerCase();
@@ -191,8 +192,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     throw new SemanticException("Unknown partition key : " + partKey);
   }
 
-  private List<String> getColumnTypes(List<String> colNames)
-      throws SemanticException{
+  private static List<String> getColumnTypes(Table tbl, List<String> colNames) {
     List<String> colTypes = new ArrayList<String>();
     List<FieldSchema> cols = tbl.getCols();
     List<String> copyColNames = new ArrayList<>(colNames);
@@ -215,21 +215,33 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     return colTypes;
   }
 
-  private String escapeBackTicks(String colName) {
+  private static String escapeBackTicks(String colName) {
     return colName.replaceAll("`", "``");
   }
 
   private String genRewrittenQuery(List<String> colNames, HiveConf conf, Map<String, String> partSpec,
-      boolean isPartitionStats) throws SemanticException{
+      boolean isPartitionStats, boolean useTableValues) throws SemanticException {
+    String rewrittenQuery = genRewrittenQuery(tbl, colNames, conf, partSpec, isPartitionStats, useTableValues);
+    isRewritten = true;
+    return rewrittenQuery;
+  }
+
+  public static String genRewrittenQuery(Table tbl, List<String> colNames, HiveConf conf, Map<String, String> partSpec,
+      boolean isPartitionStats, boolean useTableValues) throws SemanticException{
     StringBuilder rewrittenQueryBuilder = new StringBuilder("select ");
 
+    StringBuilder columnNamesBuilder = new StringBuilder();
+    StringBuilder columnDummyValuesBuilder = new StringBuilder();
     for (int i = 0; i < colNames.size(); i++) {
       if (i > 0) {
         rewrittenQueryBuilder.append(" , ");
+        columnNamesBuilder.append(" , ");
+        columnDummyValuesBuilder.append(" , ");
       }
       String func = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_STATS_NDV_ALGO).toLowerCase();
       rewrittenQueryBuilder.append("compute_stats(`");
-      rewrittenQueryBuilder.append(escapeBackTicks(colNames.get(i)));
+      final String columnName = escapeBackTicks(colNames.get(i));
+      rewrittenQueryBuilder.append(columnName);
       rewrittenQueryBuilder.append("`, '" + func + "'");
       if ("fm".equals(func)) {
         int numBitVectors = 0;
@@ -241,23 +253,45 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
         rewrittenQueryBuilder.append(", " + numBitVectors);
       }
       rewrittenQueryBuilder.append(')');
+
+      columnNamesBuilder.append("`" + columnName + "`");
+
+      columnDummyValuesBuilder.append(
+          "cast(null as " + TypeInfoUtils.getTypeInfoFromTypeString(tbl.getCols().get(i).getType()).toString() + ")");
     }
 
     if (isPartitionStats) {
       for (FieldSchema fs : tbl.getPartCols()) {
-        rewrittenQueryBuilder.append(" , `" + fs.getName() + "`");
+        final String partColumnName = " , `" + fs.getName() + "`";
+        rewrittenQueryBuilder.append(partColumnName);
+
+        columnNamesBuilder.append(partColumnName);
+
+        columnDummyValuesBuilder.append(
+            " , cast(null as " + TypeInfoUtils.getTypeInfoFromTypeString(fs.getType()).toString() + ")");
       }
     }
-    rewrittenQueryBuilder.append(" from `");
-    rewrittenQueryBuilder.append(tbl.getDbName());
-    rewrittenQueryBuilder.append("`.");
-    rewrittenQueryBuilder.append("`" + tbl.getTableName() + "`");
-    isRewritten = true;
+
+    rewrittenQueryBuilder.append(" from ");
+    if (useTableValues) {
+      //TABLE(VALUES(cast(null as int),cast(null as string))) AS tablename(col1,col2)
+      rewrittenQueryBuilder.append("table(values(");
+      // Values
+      rewrittenQueryBuilder.append(columnDummyValuesBuilder.toString());
+      rewrittenQueryBuilder.append(")) as ");
+      rewrittenQueryBuilder.append("`" + tbl.getTableName() + "`");
+      rewrittenQueryBuilder.append("(");
+      // Columns
+      rewrittenQueryBuilder.append(columnNamesBuilder.toString());
+      rewrittenQueryBuilder.append(")");
+    } else {
+      rewrittenQueryBuilder.append("`" + tbl.getDbName() + "`.`" + tbl.getTableName() + "`");
+    }
 
     // If partition level statistics is requested, add predicate and group by as needed to rewritten
     // query
     if (isPartitionStats) {
-      rewrittenQueryBuilder.append(genPartitionClause(partSpec));
+      rewrittenQueryBuilder.append(genPartitionClause(tbl, partSpec));
     }
 
     String rewrittenQuery = rewrittenQueryBuilder.toString();
@@ -311,7 +345,7 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     }
   }
 
-  private void logTypeWarning(String colName, String colType) {
+  private static void logTypeWarning(String colName, String colType) {
     String warning = "Only primitive type arguments are accepted but " + colType
         + " is passed for " + colName + ".";
     warning = "WARNING: " + warning;
@@ -353,8 +387,8 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
       } else {
         isTableLevel = true;
       }
-      colType = getColumnTypes(colNames);
-      rewrittenQuery = genRewrittenQuery(colNames, conf, partSpec, isPartitionStats);
+      colType = getColumnTypes(tbl, colNames);
+      rewrittenQuery = genRewrittenQuery(colNames, conf, partSpec, isPartitionStats, false);
       rewrittenTree = genRewrittenTree(rewrittenQuery);
     } else {
       // Not an analyze table column compute statistics statement - don't do any rewrites
@@ -391,19 +425,19 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
   /**
    * @param ast
    *          is the original analyze ast
-   * @param qb
-   *          is the qb that calls this function
-   * @param sem
-   *          is the semantic analyzer that calls this function
+   * @param context
+   *          the column stats auto gather context
    * @return
    * @throws SemanticException
    */
   public ASTNode rewriteAST(ASTNode ast, ColumnStatsAutoGatherContext context)
       throws SemanticException {
-    tbl = AnalyzeCommandUtils.getTable(ast, this);
-    colNames = getColumnName(ast);
     // Save away the original AST
     originalTree = ast;
+
+    tbl = AnalyzeCommandUtils.getTable(ast, this);
+
+    colNames = getColumnName(ast);
     boolean isPartitionStats = AnalyzeCommandUtils.isPartitionLevelStats(ast);
     Map<String, String> partSpec = null;
     checkForPartitionColumns(colNames,
@@ -414,21 +448,38 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
     }
 
     if (isPartitionStats) {
-      isTableLevel = false;
       partSpec = AnalyzeCommandUtils.getPartKeyValuePairsFromAST(tbl, ast, conf);
       handlePartialPartitionSpec(partSpec, context);
-    } else {
-      isTableLevel = true;
     }
-    colType = getColumnTypes(colNames);
-    rewrittenQuery = genRewrittenQuery(colNames, conf, partSpec, isPartitionStats);
+
+    colType = getColumnTypes(tbl, colNames);
+
+    isTableLevel = !isPartitionStats;
+
+    rewrittenQuery = genRewrittenQuery(colNames, conf, partSpec, isPartitionStats, false);
     rewrittenTree = genRewrittenTree(rewrittenQuery);
 
-    context.analyzeRewrite = new AnalyzeRewriteContext();
-    context.analyzeRewrite.setTableName(tbl.getFullyQualifiedName());
-    context.analyzeRewrite.setTblLvl(isTableLevel);
-    context.analyzeRewrite.setColName(colNames);
-    context.analyzeRewrite.setColType(colType);
     return rewrittenTree;
   }
+
+  AnalyzeRewriteContext getAnalyzeRewriteContext() {
+    AnalyzeRewriteContext analyzeRewrite = new AnalyzeRewriteContext();
+    analyzeRewrite.setTableName(tbl.getFullyQualifiedName());
+    analyzeRewrite.setTblLvl(isTableLevel);
+    analyzeRewrite.setColName(colNames);
+    analyzeRewrite.setColType(colType);
+    return analyzeRewrite;
+  }
+
+  static AnalyzeRewriteContext genAnalyzeRewriteContext(HiveConf conf, Table tbl) {
+    AnalyzeRewriteContext analyzeRewrite = new AnalyzeRewriteContext();
+    analyzeRewrite.setTableName(tbl.getFullyQualifiedName());
+    analyzeRewrite.setTblLvl(!(conf.getBoolVar(ConfVars.HIVE_STATS_COLLECT_PART_LEVEL_STATS) && tbl.isPartitioned()));
+    List<String> colNames = Utilities.getColumnNamesFromFieldSchema(tbl.getCols());
+    List<String> colTypes = getColumnTypes(tbl, colNames);
+    analyzeRewrite.setColName(colNames);
+    analyzeRewrite.setColType(colTypes);
+    return analyzeRewrite;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
index 5126a79..54f34f6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzer.java
@@ -544,7 +544,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     return ctx.getOpContext();
   }
 
-  public String genPartValueString(String partColType, String partVal) throws SemanticException {
+  public static String genPartValueString(String partColType, String partVal) throws SemanticException {
     String returnVal = partVal;
     if (partColType.equals(serdeConstants.STRING_TYPE_NAME) ||
         partColType.contains(serdeConstants.VARCHAR_TYPE_NAME) ||
@@ -7438,6 +7438,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         partitionColumnNames = viewDesc.getPartColNames();
         fileSinkColInfos = new ArrayList<>();
         destTableIsTemporary = false;
+        destTableIsMaterialization = false;
       }
 
       if (isLocal) {
@@ -7496,9 +7497,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         viewDesc.setPartCols(new ArrayList<>(partitionColumns));
       }
 
-      destTableIsTransactional = tblDesc != null && AcidUtils.isTransactionalTable(tblDesc);
-      destTableIsFullAcid = tblDesc != null && AcidUtils.isFullAcidTable(tblDesc);
-
       boolean isDestTempFile = true;
       if (!ctx.isMRTmpFileURI(destinationPath.toUri().toString())) {
         idToTableNameMap.put(String.valueOf(destTableId), destinationPath.toUri().toString());
@@ -7543,7 +7541,15 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
         tableDescriptor = PlanUtils.getTableDesc(tblDesc, cols, colTypes);
       }
 
-      boolean isDfsDir = (destType.intValue() == QBMetaData.DEST_DFS_FILE);
+      boolean isDfsDir = (destType == QBMetaData.DEST_DFS_FILE);
+
+      try {
+        destinationTable = tblDesc != null ? tblDesc.toTable(conf) : viewDesc != null ? viewDesc.toTable(conf) : null;
+      } catch (HiveException e) {
+        throw new SemanticException(e);
+      }
+
+      destTableIsFullAcid = AcidUtils.isFullAcidTable(destinationTable);
 
       if (isPartitioned) {
         // Create a SELECT that may reorder the columns if needed
@@ -7565,12 +7571,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
                 .getColumnInfos()), input), rowResolver);
         input.setColumnExprMap(colExprMap);
 
-        try {
-          destinationTable = tblDesc != null ? tblDesc.toTable(conf) : viewDesc.toTable(conf);
-        } catch (HiveException e) {
-          throw new SemanticException(e);
-        }
-
         // If this is a partitioned CTAS or MV statement, we are going to create a LoadTableDesc
         // object. Although the table does not exist in metastore, we will swamp the CreateTableTask
         // and MoveTask resulting from this LoadTable so in this specific case, first we create
@@ -7623,7 +7623,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       throw new SemanticException("Unknown destination type: " + destType);
     }
 
-    if (!(destType.intValue() == QBMetaData.DEST_DFS_FILE && qb.getIsQuery())) {
+    if (!(destType == QBMetaData.DEST_DFS_FILE && qb.getIsQuery())) {
       input = genConversionSelectOperator(dest, qb, input, tableDescriptor, dpCtx);
     }
 
@@ -7662,8 +7662,7 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     canBeMerged &= !destTableIsFullAcid;
 
     // Generate the partition columns from the parent input
-    if (destType.intValue() == QBMetaData.DEST_TABLE
-        || destType.intValue() == QBMetaData.DEST_PARTITION) {
+    if (destType == QBMetaData.DEST_TABLE || destType == QBMetaData.DEST_PARTITION) {
       genPartnCols(dest, input, qb, tableDescriptor, destinationTable, rsCtx);
     }
 
@@ -7705,14 +7704,21 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
     // and it is an insert overwrite or insert into table
     if (conf.getBoolVar(ConfVars.HIVESTATSAUTOGATHER)
         && conf.getBoolVar(ConfVars.HIVESTATSCOLAUTOGATHER)
+        && destinationTable != null && !destinationTable.isNonNative()
+        && !destTableIsTemporary && !destTableIsMaterialization
         && ColumnStatsAutoGatherContext.canRunAutogatherStats(fso)) {
-      // TODO: Column stats autogather does not work for CTAS statements
-      if (destType.intValue() == QBMetaData.DEST_TABLE && !destinationTable.isNonNative()) {
-        genAutoColumnStatsGatheringPipeline(qb, destinationTable, partSpec, input, qb.getParseInfo()
-            .isInsertIntoTable(destinationTable.getDbName(), destinationTable.getTableName()));
-      } else if (destType.intValue() == QBMetaData.DEST_PARTITION && !destinationTable.isNonNative()) {
-        genAutoColumnStatsGatheringPipeline(qb, destinationTable, destinationPartition.getSpec(), input, qb
-            .getParseInfo().isInsertIntoTable(destinationTable.getDbName(), destinationTable.getTableName()));
+      if (destType == QBMetaData.DEST_TABLE) {
+        genAutoColumnStatsGatheringPipeline(qb, destinationTable, partSpec, input,
+            qb.getParseInfo().isInsertIntoTable(destinationTable.getDbName(), destinationTable.getTableName()),
+            false);
+      } else if (destType == QBMetaData.DEST_PARTITION) {
+        genAutoColumnStatsGatheringPipeline(qb, destinationTable, destinationPartition.getSpec(), input,
+            qb.getParseInfo().isInsertIntoTable(destinationTable.getDbName(), destinationTable.getTableName()),
+            false);
+      } else if (destType == QBMetaData.DEST_LOCAL_FILE || destType == QBMetaData.DEST_DFS_FILE) {
+        // CTAS or CMV statement
+        genAutoColumnStatsGatheringPipeline(qb, destinationTable, null, input,
+            false, true);
       }
     }
     return output;
@@ -8067,13 +8073,20 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
   }
 
 
-  private void genAutoColumnStatsGatheringPipeline(QB qb, Table table,
-                                                   Map<String, String> partSpec, Operator curr, boolean isInsertInto) throws SemanticException {
+  private void genAutoColumnStatsGatheringPipeline(QB qb, Table table, Map<String, String> partSpec,
+      Operator curr, boolean isInsertInto, boolean useTableValueConstructor)
+      throws SemanticException {
     LOG.info("Generate an operator pipeline to autogather column stats for table " + table.getTableName()
         + " in query " + ctx.getCmd());
     ColumnStatsAutoGatherContext columnStatsAutoGatherContext = null;
     columnStatsAutoGatherContext = new ColumnStatsAutoGatherContext(this, conf, curr, table, partSpec, isInsertInto, ctx);
-    columnStatsAutoGatherContext.insertAnalyzePipeline();
+    if (useTableValueConstructor) {
+      // Table does not exist, use table value constructor to simulate
+      columnStatsAutoGatherContext.insertTableValuesAnalyzePipeline();
+    } else {
+      // Table already exists
+      columnStatsAutoGatherContext.insertAnalyzePipeline();
+    }
     columnStatsAutoGatherContexts.add(columnStatsAutoGatherContext);
   }
 

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
index f0f7b18..7130aba 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/CreateViewDesc.java
@@ -24,9 +24,11 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.exec.DDLTask;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -410,6 +412,11 @@ public class CreateViewDesc extends DDLDesc implements Serializable {
       }
     }
 
+    // Sets the column state for the create view statement (false since it is a creation).
+    // Similar to logic in CreateTableDesc.
+    StatsSetupConst.setStatsStateForCreateTable(tbl.getTTable().getParameters(), null,
+        StatsSetupConst.FALSE);
+
     return tbl;
   }
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q b/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q
index 00c19c7..07c3522 100644
--- a/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q
+++ b/ql/src/test/queries/clientpositive/cbo_rp_cross_product_check_2.q
@@ -4,10 +4,12 @@ set hive.cbo.returnpath.hiveop=true;
 set hive.explain.user=false;
 -- SORT_QUERY_RESULTS
 
-create table A_n18 as
+create table A_n18 (key string, value string);
+insert into A_n18
 select * from src;
 
-create table B_n14 as
+create table B_n14 (key string, value string);
+insert into B_n14
 select * from src order by key
 limit 10;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/llap_acid2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/llap_acid2.q b/ql/src/test/queries/clientpositive/llap_acid2.q
index cd06d31..84ef485 100644
--- a/ql/src/test/queries/clientpositive/llap_acid2.q
+++ b/ql/src/test/queries/clientpositive/llap_acid2.q
@@ -40,7 +40,10 @@ select cint, cbigint, cfloat, cdouble,
  cint as c1, cbigint as c2, cfloat as c3, cdouble as c4,
  cint as c8, cbigint as c7, cfloat as c6, cdouble as c5,
  cstring1, cfloat as c9, cast("1.123" as decimal(10,3))as c10,
- cast("1.123456789" as decimal(38,18)) as c11 from alltypesorc order by cdouble asc  limit 30;
+ cast("1.123456789" as decimal(38,18)) as c11
+from alltypesorc
+order by cint, cbigint, cfloat, cdouble, cstring1
+limit 30;
 
 alter table orc_llap_n2 set TBLPROPERTIES ('transactional'='true','orc.write.format'='0.12');
 
@@ -49,7 +52,10 @@ select cint, cbigint, cfloat, cdouble,
  cint as c1, cbigint as c2, cfloat as c3, cdouble as c4,
  cint as c8, cbigint as c7, cfloat as c6, cdouble as c5,
  cstring1, cfloat as c9, cast("3.321" as decimal(10,3))as c10,
- cast("9.987654321" as decimal(38,18)) as c11 from alltypesorc order by cdouble asc  limit 30;
+ cast("9.987654321" as decimal(38,18)) as c11
+from alltypesorc
+order by cint, cbigint, cfloat, cdouble, cstring1
+limit 30;
 
 
 CREATE TABLE orc_llap2 (
@@ -76,7 +82,10 @@ select cint, cbigint, cfloat, cdouble,
  cint as c1, cbigint as c2, cfloat as c3, cdouble as c4,
  cint as c8, cbigint as c7, cfloat as c6, cdouble as c5,
  cstring1, cfloat as c9, cast("1.123" as decimal(10,3))as c10,
- cast("1.123456789" as decimal(38,18)) as c11 from alltypesorc order by cdouble asc  limit 30;
+ cast("1.123456789" as decimal(38,18)) as c11
+from alltypesorc
+order by cint, cbigint, cfloat, cdouble, cstring1
+limit 30;
 
 alter table orc_llap2 set TBLPROPERTIES ('transactional'='true','orc.write.format'='0.12');
 
@@ -86,6 +95,8 @@ cdecimal2 = cast("9.987654321" as decimal(38,18))  where cstring1 = 'N016jPED08o
 
 SET hive.llap.io.enabled=true;
 
+-- SORT_QUERY_RESULTS
+
 select cstring1 from orc_llap_n2;
 select cfloat2, cint from orc_llap_n2;
 select * from orc_llap_n2;

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q
index 9735e61..3b18932 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite.q
@@ -15,8 +15,6 @@ insert into cmv_basetable_n10 values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable_n10 compute statistics for columns;
-
 create materialized view cmv_mat_view_n10
 as select a, b, c from cmv_basetable_n10 where a = 2;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_2.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_2.q
index 3f695d1..998aa49 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_2.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_2.q
@@ -13,8 +13,6 @@ insert into cmv_basetable_n9 values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable_n9 compute statistics for columns;
-
 create materialized view cmv_mat_view_n9
 as select b from cmv_basetable_n9 where c > 10.0 group by a, b, c;
 
@@ -57,8 +55,6 @@ insert into cmv_basetable_2_n4 values
  (1, 'alfred', 10.30, 2),
  (3, 'calvin', 978.76, 3);
 
-analyze table cmv_basetable_2_n4 compute statistics for columns;
-
 create materialized view cmv_mat_view_5
 as select cmv_basetable_n9.a, cmv_basetable_2_n4.c
    from cmv_basetable_n9 join cmv_basetable_2_n4 on (cmv_basetable_n9.a = cmv_basetable_2_n4.a)

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_3.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_3.q
index eb668a9..61d7c11 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_3.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_3.q
@@ -13,16 +13,12 @@ insert into cmv_basetable values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable compute statistics for columns;
-
 create table cmv_basetable_2 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into cmv_basetable_2 values
  (1, 'alfred', 10.30, 2),
  (3, 'calvin', 978.76, 3);
 
-analyze table cmv_basetable_2 compute statistics for columns;
-
 EXPLAIN
 CREATE MATERIALIZED VIEW cmv_mat_view AS
   SELECT cmv_basetable.a, cmv_basetable_2.c
@@ -51,8 +47,6 @@ GROUP BY cmv_basetable.a, cmv_basetable_2.c;
 insert into cmv_basetable_2 values
  (3, 'charlie', 15.8, 1);
 
-analyze table cmv_basetable_2 compute statistics for columns;
-
 -- CANNOT USE THE VIEW, IT IS OUTDATED
 EXPLAIN
 SELECT cmv_basetable.a
@@ -119,8 +113,6 @@ insert into cmv_irrelevant_table values
  (1, 'alfred', 10.30, 2),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_irrelevant_table compute statistics for columns;
-
 -- IT CAN STILL BE USED
 EXPLAIN
 SELECT cmv_basetable.a

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q
index f21db8a..a2754e1 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_4.q
@@ -13,16 +13,12 @@ insert into cmv_basetable_n5 values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable_n5 compute statistics for columns;
-
 create table cmv_basetable_2_n2 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into cmv_basetable_2_n2 values
  (1, 'alfred', 10.30, 2),
  (3, 'calvin', 978.76, 3);
 
-analyze table cmv_basetable_2_n2 compute statistics for columns;
-
 -- CREATE VIEW WITH REWRITE DISABLED
 EXPLAIN
 CREATE MATERIALIZED VIEW cmv_mat_view_n5 DISABLE REWRITE TBLPROPERTIES ('transactional'='true') AS
@@ -37,8 +33,6 @@ CREATE MATERIALIZED VIEW cmv_mat_view_n5 DISABLE REWRITE TBLPROPERTIES ('transac
   WHERE cmv_basetable_2_n2.c > 10.0
   GROUP BY cmv_basetable_n5.a, cmv_basetable_2_n2.c;
 
-analyze table cmv_mat_view_n5 compute statistics for columns;
-
 DESCRIBE FORMATTED cmv_mat_view_n5;
 
 -- CANNOT USE THE VIEW, IT IS DISABLED FOR REWRITE
@@ -56,8 +50,6 @@ GROUP BY cmv_basetable_n5.a, cmv_basetable_2_n2.c;
 insert into cmv_basetable_2_n2 values
  (3, 'charlie', 15.8, 1);
 
-analyze table cmv_basetable_2_n2 compute statistics for columns;
-
 -- ENABLE FOR REWRITE
 EXPLAIN
 ALTER MATERIALIZED VIEW cmv_mat_view_n5 ENABLE REWRITE;
@@ -107,6 +99,8 @@ ALTER MATERIALIZED VIEW cmv_mat_view_n5 REBUILD;
 
 ALTER MATERIALIZED VIEW cmv_mat_view_n5 REBUILD;
 
+DESCRIBE FORMATTED cmv_mat_view_n5;
+
 -- MV CAN BE USED
 EXPLAIN
 SELECT cmv_basetable_n5.a, sum(cmv_basetable_2_n2.d)
@@ -128,6 +122,8 @@ ALTER MATERIALIZED VIEW cmv_mat_view_n5 REBUILD;
 
 ALTER MATERIALIZED VIEW cmv_mat_view_n5 REBUILD;
 
+DESCRIBE FORMATTED cmv_mat_view_n5;
+
 -- MV CAN BE USED
 EXPLAIN
 SELECT cmv_basetable_n5.a, sum(cmv_basetable_2_n2.d)
@@ -150,6 +146,8 @@ ALTER MATERIALIZED VIEW cmv_mat_view_n5 REBUILD;
 
 ALTER MATERIALIZED VIEW cmv_mat_view_n5 REBUILD;
 
+DESCRIBE FORMATTED cmv_mat_view_n5;
+
 -- MV CAN BE USED
 EXPLAIN
 SELECT cmv_basetable_n5.a, sum(cmv_basetable_2_n2.d)

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q
index 3026d90..8fdc1da 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_5.q
@@ -13,28 +13,21 @@ insert into cmv_basetable_n6 values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable_n6 compute statistics for columns;
-
 create table cmv_basetable_2_n3 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into cmv_basetable_2_n3 values
  (1, 'alfred', 10.30, 2),
  (3, 'calvin', 978.76, 3);
 
-analyze table cmv_basetable_2_n3 compute statistics for columns;
-
 CREATE MATERIALIZED VIEW cmv_mat_view_n6
   TBLPROPERTIES ('transactional'='true') AS
   SELECT cmv_basetable_n6.a, cmv_basetable_2_n3.c
   FROM cmv_basetable_n6 JOIN cmv_basetable_2_n3 ON (cmv_basetable_n6.a = cmv_basetable_2_n3.a)
   WHERE cmv_basetable_2_n3.c > 10.0;
-analyze table cmv_mat_view_n6 compute statistics for columns;
 
 insert into cmv_basetable_2_n3 values
  (3, 'charlie', 15.8, 1);
 
-analyze table cmv_basetable_2_n3 compute statistics for columns;
-
 -- CANNOT USE THE VIEW, IT IS OUTDATED
 EXPLAIN
 SELECT cmv_basetable_n6.a

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q
index 8c9da8a..6978593 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_dummy.q
@@ -16,8 +16,6 @@ insert into cmv_basetable_n0 values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable_n0 compute statistics for columns;
-
 create materialized view cmv_mat_view_n0
 as select a, b, c from cmv_basetable_n0 where a = 2;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q
index 85d926f..6de65aa 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_multi_db.q
@@ -17,8 +17,6 @@ insert into cmv_basetable_n7 values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable_n7 compute statistics for columns;
-
 create database db2;
 use db2;
 

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_rebuild_dummy.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_rebuild_dummy.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_rebuild_dummy.q
index 72e3d65..e0dcf9a 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_rebuild_dummy.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_rebuild_dummy.q
@@ -13,16 +13,12 @@ insert into cmv_basetable_n1 values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable_n1 compute statistics for columns;
-
 create table cmv_basetable_2_n0 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into cmv_basetable_2_n0 values
  (1, 'alfred', 10.30, 2),
  (3, 'calvin', 978.76, 3);
 
-analyze table cmv_basetable_2_n0 compute statistics for columns;
-
 EXPLAIN
 CREATE MATERIALIZED VIEW cmv_mat_view_n1 AS
   SELECT cmv_basetable_n1.a, cmv_basetable_2_n0.c
@@ -51,8 +47,6 @@ GROUP BY cmv_basetable_n1.a, cmv_basetable_2_n0.c;
 insert into cmv_basetable_2_n0 values
  (3, 'charlie', 15.8, 1);
 
-analyze table cmv_basetable_2_n0 compute statistics for columns;
-
 -- CANNOT USE THE VIEW, IT IS OUTDATED
 EXPLAIN
 SELECT cmv_basetable_n1.a
@@ -119,8 +113,6 @@ insert into cmv_irrelevant_table_n0 values
  (1, 'alfred', 10.30, 2),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_irrelevant_table_n0 compute statistics for columns;
-
 -- IT CAN STILL BE USED
 EXPLAIN
 SELECT cmv_basetable_n1.a

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q
index 4cdb715..fa07401 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window.q
@@ -12,16 +12,12 @@ insert into cmv_basetable_n3 values
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
 
-analyze table cmv_basetable_n3 compute statistics for columns;
-
 create table cmv_basetable_2_n1 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true');
 
 insert into cmv_basetable_2_n1 values
  (1, 'alfred', 10.30, 2),
  (3, 'calvin', 978.76, 3);
 
-analyze table cmv_basetable_2_n1 compute statistics for columns;
-
 -- CREATE VIEW WITH REWRITE DISABLED
 EXPLAIN
 CREATE MATERIALIZED VIEW cmv_mat_view_n3 DISABLE REWRITE TBLPROPERTIES('rewriting.time.window'='5min') AS
@@ -53,8 +49,6 @@ GROUP BY cmv_basetable_n3.a, cmv_basetable_2_n1.c;
 insert into cmv_basetable_2_n1 values
  (3, 'charlie', 15.8, 1);
 
-analyze table cmv_basetable_2_n1 compute statistics for columns;
-
 -- ENABLE FOR REWRITE
 EXPLAIN
 ALTER MATERIALIZED VIEW cmv_mat_view_n3 ENABLE REWRITE;

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q
index 6873673..418ec45 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_create_rewrite_time_window_2.q
@@ -9,13 +9,11 @@ insert into cmv_basetable_n100 values
  (2, 'bonnie', 172342.2, 3),
  (3, 'calvin', 978.76, 3),
  (3, 'charlie', 9.8, 1);
-analyze table cmv_basetable_n100 compute statistics for columns;
 
 create table cmv_basetable_2_n100 (a int, b varchar(256), c decimal(10,2), d int) stored as orc TBLPROPERTIES ('transactional'='true');
 insert into cmv_basetable_2_n100 values
  (1, 'alfred', 10.30, 2),
  (3, 'calvin', 978.76, 3);
-analyze table cmv_basetable_2_n100 compute statistics for columns;
 
 -- CREATE MATERIALIZED VIEW
 CREATE MATERIALIZED VIEW cmv_mat_view_n300 AS
@@ -29,7 +27,6 @@ DESCRIBE FORMATTED cmv_mat_view_n300;
 
 insert into cmv_basetable_2_n100 values
  (3, 'charlie', 15.8, 1);
-analyze table cmv_basetable_2_n100 compute statistics for columns;
 
 -- OUTDATED: YES
 DESCRIBE FORMATTED cmv_mat_view_n300;

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_rewrite_1.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_rewrite_1.q b/ql/src/test/queries/clientpositive/materialized_view_rewrite_1.q
index 18b9f7d..ff7cefc 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_1.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_1.q
@@ -15,7 +15,6 @@ create table emps_n3 (
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into emps_n3 values (100, 10, 'Bill', 10000, 1000), (200, 20, 'Eric', 8000, 500),
   (150, 10, 'Sebastian', 7000, null), (110, 10, 'Theodore', 10000, 250), (120, 10, 'Bill', 10000, 250);
-analyze table emps_n3 compute statistics for columns;
 
 create table depts_n2 (
   deptno int,
@@ -23,21 +22,18 @@ create table depts_n2 (
   locationid int)
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into depts_n2 values (10, 'Sales', 10), (30, 'Marketing', null), (20, 'HR', 20);
-analyze table depts_n2 compute statistics for columns;
 
 create table dependents_n2 (
   empid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into dependents_n2 values (10, 'Michael'), (20, 'Jane');
-analyze table dependents_n2 compute statistics for columns;
 
 create table locations_n2 (
   locationid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into locations_n2 values (10, 'San Francisco'), (20, 'San Diego');
-analyze table locations_n2 compute statistics for columns;
 
 alter table emps_n3 add constraint pk1 primary key (empid) disable novalidate rely;
 alter table depts_n2 add constraint pk2 primary key (deptno) disable novalidate rely;
@@ -50,7 +46,6 @@ alter table depts_n2 add constraint fk2 foreign key (locationid) references loca
 -- EXAMPLE 1
 create materialized view mv1_n2 as
 select * from emps_n3 where empid < 150;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select *
@@ -67,7 +62,6 @@ drop materialized view mv1_n2;
 create materialized view mv1_n2 as
 select deptno, name, salary, commission
 from emps_n3;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select emps_n3.name, emps_n3.salary, emps_n3.commission
@@ -84,7 +78,6 @@ drop materialized view mv1_n2;
 create materialized view mv1_n2 as
 select empid deptno from emps_n3
 join depts_n2 using (deptno);
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select empid deptno from emps_n3
@@ -98,7 +91,6 @@ drop materialized view mv1_n2;
 -- EXAMPLE 4
 create materialized view mv1_n2 as
 select * from emps_n3 where empid < 200;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select * from emps_n3 where empid > 120
@@ -112,7 +104,6 @@ drop materialized view mv1_n2;
 -- EXAMPLE 5 - NO MV, ALREADY UNIQUE
 create materialized view mv1_n2 as
 select empid, deptno from emps_n3 group by empid, deptno;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select empid, deptno from emps_n3 group by empid, deptno;
@@ -124,7 +115,6 @@ drop materialized view mv1_n2;
 -- EXAMPLE 5 - NO MV, ALREADY UNIQUE
 create materialized view mv1_n2 as
 select empid, name from emps_n3 group by empid, name;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select empid, name from emps_n3 group by empid, name;
@@ -136,7 +126,6 @@ drop materialized view mv1_n2;
 -- EXAMPLE 5
 create materialized view mv1_n2 as
 select name, salary from emps_n3 group by name, salary;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select name, salary from emps_n3 group by name, salary;
@@ -148,7 +137,6 @@ drop materialized view mv1_n2;
 -- EXAMPLE 6
 create materialized view mv1_n2 as
 select name, salary from emps_n3 group by name, salary;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select name from emps_n3 group by name;
@@ -160,7 +148,6 @@ drop materialized view mv1_n2;
 -- EXAMPLE 7
 create materialized view mv1_n2 as
 select name, salary from emps_n3 where deptno = 10 group by name, salary;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select name from emps_n3 where deptno = 10 group by name;
@@ -173,7 +160,6 @@ drop materialized view mv1_n2;
 create materialized view mv1_n2 as
 select name, salary, count(*) as c, sum(empid) as s
 from emps_n3 group by name, salary;
-analyze table mv1_n2 compute statistics for columns;
 
 explain
 select name from emps_n3 group by name;

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_rewrite_10.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_rewrite_10.q b/ql/src/test/queries/clientpositive/materialized_view_rewrite_10.q
index 9542792..ff593a7 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_10.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_10.q
@@ -14,13 +14,11 @@ create table emps_n10 (
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into emps_n10 values (100, 10, 'Bill', 10000, 1000), (200, 20, 'Eric', 8000, 500),
   (150, 10, 'Sebastian', 7000, null), (110, 10, 'Theodore', 10000, 250), (110, 10, 'Bill', 10000, 250);
-analyze table emps_n10 compute statistics for columns;
 
 -- EXAMPLE 1
 create materialized view mv1_n10 as
 select deptno, sum(salary), count(salary) as a
 from emps_n10 group by deptno;
-analyze table mv1_n10 compute statistics for columns;
 
 explain
 select deptno, avg(salary) as a
@@ -35,7 +33,6 @@ drop materialized view mv1_n10;
 create materialized view mv1_n10 as
 select salary, sum(salary), count(salary) as a
 from emps_n10 group by salary;
-analyze table mv1_n10 compute statistics for columns;
 
 explain
 select salary, avg(salary) as a
@@ -50,7 +47,6 @@ drop materialized view mv1_n10;
 create materialized view mv1_n10 as
 select salary, sum(salary), count(salary) as a
 from emps_n10 where salary > 0 group by salary;
-analyze table mv1_n10 compute statistics for columns;
 
 explain
 select salary, avg(salary) as a
@@ -71,12 +67,10 @@ create table emps_n10_2 (
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into emps_n10_2 values (100, 10, 'Bill', 1, 1000), (200, 20, 'Eric', 2, 500),
   (150, 10, 'Sebastian', 2, null), (110, 10, 'Theodore', 3, 250), (110, 10, 'Bill', 0, 250);
-analyze table emps_n10_2 compute statistics for columns;
 
 create materialized view mv1_n10 as
 select salary, sum(salary), count(salary) as a
 from emps_n10_2 where salary > 0 group by salary;
-analyze table mv1_n10 compute statistics for columns;
 
 explain
 select avg(salary)

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_rewrite_2.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_rewrite_2.q b/ql/src/test/queries/clientpositive/materialized_view_rewrite_2.q
index 3a447fc..feba84e 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_2.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_2.q
@@ -15,7 +15,6 @@ create table emps_n0 (
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into emps_n0 values (100, 10, 'Bill', 10000, 1000), (200, 20, 'Eric', 8000, 500),
   (150, 10, 'Sebastian', 7000, null), (110, 10, 'Theodore', 10000, 250), (110, 10, 'Bill', 10000, 250);
-analyze table emps_n0 compute statistics for columns;
 
 create table depts_n0 (
   deptno int,
@@ -23,21 +22,18 @@ create table depts_n0 (
   locationid int)
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into depts_n0 values (10, 'Sales', 10), (30, 'Marketing', null), (20, 'HR', 20);
-analyze table depts_n0 compute statistics for columns;
 
 create table dependents_n0 (
   empid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into dependents_n0 values (10, 'Michael'), (10, 'Jane');
-analyze table dependents_n0 compute statistics for columns;
 
 create table locations_n0 (
   locationid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into locations_n0 values (10, 'San Francisco'), (10, 'San Diego');
-analyze table locations_n0 compute statistics for columns;
 
 alter table emps_n0 add constraint pk1 primary key (empid) disable novalidate rely;
 alter table depts_n0 add constraint pk2 primary key (deptno) disable novalidate rely;
@@ -52,7 +48,6 @@ create materialized view mv1_n0 as
 select empid, depts_n0.deptno from emps_n0
 join depts_n0 using (deptno) where depts_n0.deptno > 10
 group by empid, depts_n0.deptno;
-analyze table mv1_n0 compute statistics for columns;
 
 explain
 select empid from emps_n0
@@ -70,7 +65,6 @@ create materialized view mv1_n0 as
 select depts_n0.deptno, empid from depts_n0
 join emps_n0 using (deptno) where depts_n0.deptno > 10
 group by empid, depts_n0.deptno;
-analyze table mv1_n0 compute statistics for columns;
 
 explain
 select empid from emps_n0
@@ -88,7 +82,6 @@ create materialized view mv1_n0 as
 select empid, depts_n0.deptno from emps_n0
 join depts_n0 using (deptno) where emps_n0.deptno > 10
 group by empid, depts_n0.deptno;
-analyze table mv1_n0 compute statistics for columns;
 
 explain
 select empid from emps_n0
@@ -106,7 +99,6 @@ create materialized view mv1_n0 as
 select depts_n0.deptno, emps_n0.empid from depts_n0
 join emps_n0 using (deptno) where emps_n0.empid > 10
 group by depts_n0.deptno, emps_n0.empid;
-analyze table mv1_n0 compute statistics for columns;
 
 explain
 select depts_n0.deptno from depts_n0
@@ -124,7 +116,6 @@ create materialized view mv1_n0 as
 select depts_n0.deptno, emps_n0.empid from depts_n0
 join emps_n0 using (deptno) where emps_n0.empid > 10
 group by depts_n0.deptno, emps_n0.empid;
-analyze table mv1_n0 compute statistics for columns;
 
 explain
 select depts_n0.deptno from depts_n0
@@ -143,7 +134,6 @@ select depts_n0.name, dependents_n0.name as name2, emps_n0.deptno, depts_n0.dept
 from depts_n0, dependents_n0, emps_n0
 where depts_n0.deptno > 10
 group by depts_n0.name, dependents_n0.name, emps_n0.deptno, depts_n0.deptno, dependents_n0.empid;
-analyze table mv1_n0 compute statistics for columns;
 
 explain
 select dependents_n0.empid

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_rewrite_3.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_rewrite_3.q b/ql/src/test/queries/clientpositive/materialized_view_rewrite_3.q
index 0823f59..ff02ceb 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_3.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_3.q
@@ -15,7 +15,6 @@ create table emps_n9 (
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into emps_n9 values (100, 10, 'Bill', 10000, 1000), (200, 20, 'Eric', 8000, 500),
   (150, 10, 'Sebastian', 7000, null), (120, 10, 'Theodore', 10000, 250);
-analyze table emps_n9 compute statistics for columns;
 
 create table depts_n7 (
   deptno int,
@@ -23,21 +22,18 @@ create table depts_n7 (
   locationid int)
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into depts_n7 values (10, 'Sales', 10), (30, 'Marketing', null), (20, 'HR', 20);
-analyze table depts_n7 compute statistics for columns;
 
 create table dependents_n5 (
   empid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into dependents_n5 values (10, 'Michael'), (20, 'Jane');
-analyze table dependents_n5 compute statistics for columns;
 
 create table locations_n5 (
   locationid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into locations_n5 values (10, 'San Francisco'), (20, 'San Diego');
-analyze table locations_n5 compute statistics for columns;
 
 alter table emps_n9 add constraint pk1 primary key (empid) disable novalidate rely;
 alter table depts_n7 add constraint pk2 primary key (deptno) disable novalidate rely;
@@ -51,7 +47,6 @@ alter table depts_n7 add constraint fk2 foreign key (locationid) references loca
 create materialized view mv1_n5 as
 select empid deptno from emps_n9
 join depts_n7 using (deptno);
-analyze table mv1_n5 compute statistics for columns;
 
 explain
 select empid deptno from emps_n9
@@ -66,7 +61,6 @@ drop materialized view mv1_n5;
 create materialized view mv1_n5 as
 select cast(empid as BIGINT) from emps_n9
 join depts_n7 using (deptno);
-analyze table mv1_n5 compute statistics for columns;
 
 explain
 select empid deptno from emps_n9
@@ -81,7 +75,6 @@ drop materialized view mv1_n5;
 create materialized view mv1_n5 as
 select cast(empid as BIGINT) from emps_n9
 join depts_n7 using (deptno);
-analyze table mv1_n5 compute statistics for columns;
 
 explain
 select empid deptno from emps_n9
@@ -97,7 +90,6 @@ create materialized view mv1_n5 as
 select depts_n7.name
 from emps_n9
 join depts_n7 on (emps_n9.deptno = depts_n7.deptno);
-analyze table mv1_n5 compute statistics for columns;
 
 explain
 select dependents_n5.empid

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_rewrite_4.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_rewrite_4.q b/ql/src/test/queries/clientpositive/materialized_view_rewrite_4.q
index 6724cec..08c1956 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_4.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_4.q
@@ -15,7 +15,6 @@ create table emps_n5 (
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into emps_n5 values (100, 10, 'Bill', 10000, 1000), (200, 20, 'Eric', 8000, 500),
   (150, 10, 'Sebastian', 7000, null), (110, 10, 'Theodore', 10000, 250), (120, 10, 'Bill', 10000, 250);
-analyze table emps_n5 compute statistics for columns;
 
 create table depts_n4 (
   deptno int,
@@ -23,21 +22,18 @@ create table depts_n4 (
   locationid int)
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into depts_n4 values (10, 'Sales', 10), (30, 'Marketing', null), (20, 'HR', 20);
-analyze table depts_n4 compute statistics for columns;
 
 create table dependents_n3 (
   empid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into dependents_n3 values (10, 'Michael'), (20, 'Jane');
-analyze table dependents_n3 compute statistics for columns;
 
 create table locations_n3 (
   locationid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into locations_n3 values (10, 'San Francisco'), (20, 'San Diego');
-analyze table locations_n3 compute statistics for columns;
 
 alter table emps_n5 add constraint pk1 primary key (empid) disable novalidate rely;
 alter table depts_n4 add constraint pk2 primary key (deptno) disable novalidate rely;
@@ -52,7 +48,6 @@ alter table depts_n4 add constraint fk2 foreign key (locationid) references loca
 create materialized view mv1_n3 as
 select name, salary, count(*) as c, sum(empid) as s
 from emps_n5 group by name, salary;
-analyze table mv1_n3 compute statistics for columns;
 
 explain
 select name, count(*) as c, sum(empid) as s
@@ -67,7 +62,6 @@ drop materialized view mv1_n3;
 create materialized view mv1_n3 as
 select name, salary, count(*) as c, sum(empid) as s
 from emps_n5 group by name, salary;
-analyze table mv1_n3 compute statistics for columns;
 
 explain
 select salary, name, sum(empid) as s, count(*) as c
@@ -84,7 +78,6 @@ create materialized view mv1_n3 as
 select empid, emps_n5.deptno, count(*) as c, sum(empid) as s
 from emps_n5 join depts_n4 using (deptno)
 group by empid, emps_n5.deptno;
-analyze table mv1_n3 compute statistics for columns;
 
 explain
 select depts_n4.deptno, count(*) as c, sum(empid) as s
@@ -102,7 +95,6 @@ create materialized view mv1_n3 as
 select empid, emps_n5.deptno, count(*) as c, sum(empid) as s
 from emps_n5 join depts_n4 using (deptno)
 where emps_n5.deptno >= 10 group by empid, emps_n5.deptno;
-analyze table mv1_n3 compute statistics for columns;
 
 explain
 select depts_n4.deptno, sum(empid) as s
@@ -120,7 +112,6 @@ create materialized view mv1_n3 as
 select empid, depts_n4.deptno, count(*) + 1 as c, sum(empid) as s
 from emps_n5 join depts_n4 using (deptno)
 where depts_n4.deptno >= 10 group by empid, depts_n4.deptno;
-analyze table mv1_n3 compute statistics for columns;
 
 explain
 select depts_n4.deptno, sum(empid) + 1 as s
@@ -139,7 +130,6 @@ select depts_n4.name, sum(salary) as s
 from emps_n5
 join depts_n4 on (emps_n5.deptno = depts_n4.deptno)
 group by depts_n4.name;
-analyze table mv1_n3 compute statistics for columns;
 
 explain
 select dependents_n3.empid, sum(salary) as s
@@ -162,7 +152,6 @@ select dependents_n3.empid, emps_n5.deptno, count(distinct salary) as s
 from emps_n5
 join dependents_n3 on (emps_n5.empid = dependents_n3.empid)
 group by dependents_n3.empid, emps_n5.deptno;
-analyze table mv1_n3 compute statistics for columns;
 
 explain
 select emps_n5.deptno, count(distinct salary) as s

http://git-wip-us.apache.org/repos/asf/hive/blob/138b00ca/ql/src/test/queries/clientpositive/materialized_view_rewrite_5.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/materialized_view_rewrite_5.q b/ql/src/test/queries/clientpositive/materialized_view_rewrite_5.q
index d87928c..1e9a9b9 100644
--- a/ql/src/test/queries/clientpositive/materialized_view_rewrite_5.q
+++ b/ql/src/test/queries/clientpositive/materialized_view_rewrite_5.q
@@ -15,7 +15,6 @@ create table emps_n2 (
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into emps_n2 values (100, 10, 'Bill', 10000, 1000), (200, 20, 'Eric', 8000, 500),
   (150, 10, 'Sebastian', 7000, null), (110, 10, 'Theodore', 10000, 250), (120, 10, 'Bill', 10000, 250);
-analyze table emps_n2 compute statistics for columns;
 
 create table depts_n1 (
   deptno int,
@@ -23,21 +22,18 @@ create table depts_n1 (
   locationid int)
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into depts_n1 values (10, 'Sales', 10), (30, 'Marketing', null), (20, 'HR', 20);
-analyze table depts_n1 compute statistics for columns;
 
 create table dependents_n1 (
   empid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into dependents_n1 values (10, 'Michael'), (20, 'Jane');
-analyze table dependents_n1 compute statistics for columns;
 
 create table locations_n1 (
   locationid int,
   name varchar(256))
 stored as orc TBLPROPERTIES ('transactional'='true');
 insert into locations_n1 values (10, 'San Francisco'), (20, 'San Diego');
-analyze table locations_n1 compute statistics for columns;
 
 alter table emps_n2 add constraint pk1 primary key (empid) disable novalidate rely;
 alter table depts_n1 add constraint pk2 primary key (deptno) disable novalidate rely;
@@ -54,7 +50,6 @@ alter table depts_n1 change column locationid locationid int constraint nn2 not
 -- EXAMPLE 8
 create materialized view mv1_n1 as
 select name, deptno, salary from emps_n2 where deptno > 15 group by name, deptno, salary;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select name from emps_n2 where deptno >= 20 group by name;
@@ -67,7 +62,6 @@ drop materialized view mv1_n1;
 create materialized view mv1_n1 as
 select name, deptno, salary, count(*) as c, sum(empid) as s
 from emps_n2 where deptno >= 15 group by name, deptno, salary;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select name, sum(empid) as s
@@ -87,7 +81,6 @@ join locations_n1 on (locations_n1.name = dependents_n1.name)
 join emps_n2 on (emps_n2.deptno = depts_n1.deptno)
 where depts_n1.deptno > 10 and depts_n1.deptno < 20
 group by depts_n1.deptno, dependents_n1.empid;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select dependents_n1.empid
@@ -113,7 +106,6 @@ create materialized view mv1_n1 as
 select empid, depts_n1.deptno, count(*) as c, sum(empid) as s
 from emps_n2 join depts_n1 using (deptno)
 group by empid, depts_n1.deptno;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select deptno from emps_n2 group by deptno;
@@ -127,7 +119,6 @@ create materialized view mv1_n1 as
 select empid, depts_n1.deptno, count(*) as c, sum(empid) as s
 from emps_n2 join depts_n1 using (deptno)
 group by empid, depts_n1.deptno;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select deptno, empid, sum(empid) as s, count(*) as c
@@ -144,7 +135,6 @@ select dependents_n1.empid, emps_n2.deptno, sum(salary) as s
 from emps_n2
 join dependents_n1 on (emps_n2.empid = dependents_n1.empid)
 group by dependents_n1.empid, emps_n2.deptno;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select dependents_n1.empid, sum(salary) as s
@@ -167,7 +157,6 @@ select dependents_n1.empid, emps_n2.deptno, sum(salary) as s
 from emps_n2
 join dependents_n1 on (emps_n2.empid = dependents_n1.empid)
 group by dependents_n1.empid, emps_n2.deptno;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select depts_n1.name, sum(salary) as s
@@ -190,7 +179,6 @@ select a.empid deptno from
 (select * from emps_n2 where empid = 1) a
 join depts_n1 on (a.deptno = depts_n1.deptno)
 join dependents_n1 on (a.empid = dependents_n1.empid);
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select a.empid from 
@@ -209,7 +197,6 @@ select a.empid, a.deptno from
 (select * from emps_n2 where empid = 1) a
 join depts_n1 on (a.deptno = depts_n1.deptno)
 join dependents_n1 on (a.empid = dependents_n1.empid);
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select a.empid from 
@@ -227,7 +214,6 @@ create materialized view mv1_n1 as
 select empid deptno from
 (select * from emps_n2 where empid = 1) a
 join depts_n1 on (a.deptno = depts_n1.deptno);
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select empid from emps_n2 where empid = 1;
@@ -242,7 +228,6 @@ select emps_n2.empid, emps_n2.deptno from emps_n2
 join depts_n1 on (emps_n2.deptno = depts_n1.deptno)
 join dependents_n1 on (emps_n2.empid = dependents_n1.empid)
 where emps_n2.empid = 1;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select emps_n2.empid from emps_n2
@@ -262,7 +247,6 @@ join depts_n1 a on (emps_n2.deptno=a.deptno)
 join depts_n1 b on (emps_n2.deptno=b.deptno)
 join dependents_n1 on (emps_n2.empid = dependents_n1.empid)
 where emps_n2.empid = 1;
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select emps_n2.empid from emps_n2
@@ -282,7 +266,6 @@ join depts_n1 a on (emps_n2.deptno=a.deptno)
 join depts_n1 b on (emps_n2.deptno=b.deptno)
 join dependents_n1 on (emps_n2.empid = dependents_n1.empid)
 where emps_n2.name = 'Sebastian';
-analyze table mv1_n1 compute statistics for columns;
 
 explain
 select emps_n2.empid from emps_n2