You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by kg...@apache.org on 2020/05/04 14:12:44 UTC

[hive] branch master updated: HIVE-21304: Make bucketing version usage more robust (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new c34ee9d  HIVE-21304: Make bucketing version usage more robust (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
c34ee9d is described below

commit c34ee9d79bf6931a92b61af98a6c8f09c6b9ad73
Author: Zoltan Haindrich <ki...@rxd.hu>
AuthorDate: Mon May 4 13:35:12 2020 +0000

    HIVE-21304: Make bucketing version usage more robust (Zoltan Haindrich reviewed by Jesus Camacho Rodriguez)
    
    Signed-off-by: Zoltan Haindrich <zh...@cloudera.com>
---
 .../insert_into_dynamic_partitions.q.out           |   3 +
 .../results/clientpositive/insert_into_table.q.out |   7 +
 .../insert_overwrite_directory.q.out               |  30 +
 .../insert_overwrite_dynamic_partitions.q.out      |   3 +
 .../clientpositive/insert_overwrite_table.q.out    |   7 +
 .../write_final_output_blobstore.q.out             |  16 +
 .../test/resources/testconfiguration.properties    |   1 +
 .../org/apache/hadoop/hive/ql/exec/JoinUtil.java   |  22 +-
 .../org/apache/hadoop/hive/ql/exec/Operator.java   |   9 -
 .../hadoop/hive/ql/exec/OperatorFactory.java       |  10 +-
 .../hadoop/hive/ql/exec/ReduceSinkOperator.java    |   4 +-
 .../org/apache/hadoop/hive/ql/exec/Utilities.java  |  12 +-
 .../hive/ql/exec/persistence/PTFRowContainer.java  |   2 +
 .../VectorReduceSinkObjectHashOperator.java        |  16 +-
 .../hive/ql/optimizer/AbstractSMBJoinProc.java     |   6 +-
 .../hive/ql/optimizer/BucketVersionPopulator.java  | 229 ++++++
 .../hive/ql/optimizer/ConvertJoinMapJoin.java      |  30 +-
 .../apache/hadoop/hive/ql/optimizer/Optimizer.java |   2 +
 .../ql/optimizer/SortedDynPartitionOptimizer.java  |   4 +-
 ...SortedDynPartitionTimeGranularityOptimizer.java |   1 +
 .../opconventer/HiveTableScanVisitor.java          |   1 -
 .../correlation/ReduceSinkDeDuplicationUtils.java  |  12 +-
 .../annotation/OpTraitsRulesProcFactory.java       |  33 +-
 .../hive/ql/optimizer/physical/Vectorizer.java     |   5 -
 .../ql/optimizer/spark/SparkMapJoinOptimizer.java  |   2 +-
 .../hadoop/hive/ql/parse/BaseSemanticAnalyzer.java |   2 +
 .../hadoop/hive/ql/parse/SemanticAnalyzer.java     |   3 -
 .../apache/hadoop/hive/ql/parse/TezCompiler.java   |  38 +-
 .../hadoop/hive/ql/parse/spark/SparkCompiler.java  |  43 -
 .../hadoop/hive/ql/plan/AbstractOperatorDesc.java  |  10 +
 .../apache/hadoop/hive/ql/plan/FileSinkDesc.java   |   9 +-
 .../org/apache/hadoop/hive/ql/plan/OpTraits.java   |  15 +-
 .../apache/hadoop/hive/ql/plan/OperatorDesc.java   |   4 +
 .../org/apache/hadoop/hive/ql/plan/PlanUtils.java  |   1 +
 .../apache/hadoop/hive/ql/plan/ReduceSinkDesc.java |   6 +
 .../apache/hadoop/hive/ql/plan/TableScanDesc.java  |   1 +
 .../apache/hadoop/hive/ql/exec/TestUtilities.java  |   3 +-
 .../clientpositive/infer_bucket_sort_num_buckets.q |   2 +-
 .../queries/clientpositive/murmur_hash_migration.q |  42 +-
 .../clientpositive/murmur_hash_migration2.q        |  44 ++
 .../clientpositive/beeline/smb_mapjoin_1.q.out     |  24 +
 .../clientpositive/beeline/smb_mapjoin_10.q.out    |   3 +
 .../clientpositive/beeline/smb_mapjoin_11.q.out    |  23 +
 .../clientpositive/beeline/smb_mapjoin_12.q.out    |  23 +
 .../clientpositive/beeline/smb_mapjoin_13.q.out    |  14 +
 .../clientpositive/beeline/smb_mapjoin_16.q.out    |   5 +
 .../clientpositive/beeline/smb_mapjoin_2.q.out     |  24 +
 .../clientpositive/beeline/smb_mapjoin_3.q.out     |  24 +
 .../clientpositive/beeline/smb_mapjoin_7.q.out     |   8 +
 .../clientpositive/binary_output_format.q.out      |   8 +
 .../results/clientpositive/bucket_map_join_1.q.out |   4 +
 .../results/clientpositive/bucket_map_join_2.q.out |   4 +
 .../clientpositive/bucket_map_join_spark1.q.out    |  12 +
 .../clientpositive/bucket_map_join_spark2.q.out    |  12 +
 .../clientpositive/bucket_map_join_spark3.q.out    |  12 +
 .../clientpositive/bucket_map_join_spark4.q.out    |   4 +
 .../results/clientpositive/bucketcontext_1.q.out   |  13 +
 .../results/clientpositive/bucketcontext_2.q.out   |  13 +
 .../results/clientpositive/bucketcontext_3.q.out   |  13 +
 .../results/clientpositive/bucketcontext_4.q.out   |  13 +
 .../results/clientpositive/bucketcontext_5.q.out   |  13 +
 .../results/clientpositive/bucketcontext_6.q.out   |  13 +
 .../results/clientpositive/bucketcontext_7.q.out   |  13 +
 .../results/clientpositive/bucketcontext_8.q.out   |  13 +
 .../results/clientpositive/bucketmapjoin10.q.out   |   4 +
 .../results/clientpositive/bucketmapjoin11.q.out   |   8 +
 .../results/clientpositive/bucketmapjoin12.q.out   |   8 +
 .../results/clientpositive/bucketmapjoin13.q.out   |  16 +
 .../results/clientpositive/bucketmapjoin5.q.out    |  14 +
 .../results/clientpositive/bucketmapjoin8.q.out    |   8 +
 .../results/clientpositive/bucketmapjoin9.q.out    |   8 +
 .../clientpositive/bucketmapjoin_negative.q.out    |   7 +
 .../clientpositive/bucketmapjoin_negative2.q.out   |   7 +
 .../clientpositive/bucketmapjoin_negative3.q.out   |  18 +
 .../results/clientpositive/cbo_rp_auto_join1.q.out |  48 ++
 .../druid/druidkafkamini_basic.q.out               |   6 +
 .../druid/druidmini_expressions.q.out              |   4 +
 .../encryption_join_unencrypted_tbl.q.out          |   6 +
 ...ption_join_with_different_encryption_keys.q.out |   6 +
 .../results/clientpositive/explain_rearrange.q.out |  76 ++
 .../results/clientpositive/groupby_map_ppr.q.out   |   8 +
 .../groupby_map_ppr_multi_distinct.q.out           |   8 +
 .../test/results/clientpositive/groupby_ppr.q.out  |   8 +
 .../groupby_ppr_multi_distinct.q.out               |  16 +
 .../results/clientpositive/groupby_sort_1_23.q.out | 138 ++++
 .../results/clientpositive/groupby_sort_6.q.out    |  24 +
 .../clientpositive/groupby_sort_skew_1_23.q.out    | 159 ++++
 .../infer_bucket_sort_dyn_part.q.out               |   2 +-
 .../infer_bucket_sort_map_operators.q.out          |  21 +
 .../infer_bucket_sort_num_buckets.q.out            | 269 ++++++-
 .../infer_bucket_sort_reducers_power_two.q.out     |  12 +-
 ql/src/test/results/clientpositive/input23.q.out   |   6 +
 ql/src/test/results/clientpositive/input42.q.out   |   4 +
 .../test/results/clientpositive/input_part1.q.out  |   7 +
 .../test/results/clientpositive/input_part2.q.out  |  13 +
 .../test/results/clientpositive/input_part7.q.out  |   6 +
 .../test/results/clientpositive/input_part9.q.out  |   2 +
 ql/src/test/results/clientpositive/join17.q.out    |  10 +
 ql/src/test/results/clientpositive/join26.q.out    |   6 +
 ql/src/test/results/clientpositive/join32.q.out    |   6 +
 ql/src/test/results/clientpositive/join33.q.out    |   6 +
 ql/src/test/results/clientpositive/join34.q.out    |   8 +
 ql/src/test/results/clientpositive/join35.q.out    |  14 +
 ql/src/test/results/clientpositive/join9.q.out     |  10 +
 .../clientpositive/join_filters_overlap.q.out      |  65 ++
 .../test/results/clientpositive/join_map_ppr.q.out |  14 +
 .../kafka/kafka_storage_handler.q.out              |   8 +
 .../results/clientpositive/list_bucket_dml_1.q.out |   7 +
 .../clientpositive/list_bucket_dml_11.q.out        |   7 +
 .../clientpositive/list_bucket_dml_12.q.out        |   9 +
 .../clientpositive/list_bucket_dml_13.q.out        |   7 +
 .../clientpositive/list_bucket_dml_14.q.out        |   7 +
 .../results/clientpositive/list_bucket_dml_2.q.out |   7 +
 .../results/clientpositive/list_bucket_dml_3.q.out |   7 +
 .../results/clientpositive/list_bucket_dml_4.q.out |  12 +
 .../results/clientpositive/list_bucket_dml_5.q.out |   7 +
 .../results/clientpositive/list_bucket_dml_6.q.out |  12 +
 .../results/clientpositive/list_bucket_dml_7.q.out |  12 +
 .../results/clientpositive/list_bucket_dml_8.q.out |   7 +
 .../results/clientpositive/list_bucket_dml_9.q.out |  12 +
 .../list_bucket_query_multiskew_1.q.out            |   8 +
 .../list_bucket_query_multiskew_2.q.out            |   6 +
 .../list_bucket_query_multiskew_3.q.out            |   6 +
 .../list_bucket_query_oneskew_1.q.out              |   6 +
 .../list_bucket_query_oneskew_2.q.out              |  12 +
 .../list_bucket_query_oneskew_3.q.out              |   2 +
 .../clientpositive/llap/acid_bucket_pruning.q.out  |   2 +
 .../clientpositive/llap/acid_nullscan.q.out        |   4 +
 .../clientpositive/llap/autoColumnStats_5a.q.out   |  10 +
 .../clientpositive/llap/autoColumnStats_8.q.out    |  10 +
 .../llap/auto_join_reordering_values.q.out         |  18 +
 .../clientpositive/llap/auto_smb_mapjoin_14.q.out  |  13 +
 .../llap/auto_sortmerge_join_1.q.out               |  15 +
 .../llap/auto_sortmerge_join_10.q.out              |   1 +
 .../llap/auto_sortmerge_join_11.q.out              |  27 +
 .../llap/auto_sortmerge_join_12.q.out              |  13 +
 .../llap/auto_sortmerge_join_14.q.out              |   2 +
 .../llap/auto_sortmerge_join_15.q.out              |   2 +
 .../llap/auto_sortmerge_join_16.q.out              |   2 +
 .../llap/auto_sortmerge_join_2.q.out               |  10 +
 .../llap/auto_sortmerge_join_3.q.out               |  15 +
 .../llap/auto_sortmerge_join_4.q.out               |  15 +
 .../llap/auto_sortmerge_join_5.q.out               |  15 +
 .../llap/auto_sortmerge_join_6.q.out               |   4 +
 .../llap/auto_sortmerge_join_7.q.out               |  15 +
 .../llap/auto_sortmerge_join_8.q.out               |  15 +
 .../llap/auto_sortmerge_join_9.q.out               |  26 +
 .../test/results/clientpositive/llap/bucket1.q.out |   7 +
 .../test/results/clientpositive/llap/bucket2.q.out |   5 +
 .../test/results/clientpositive/llap/bucket3.q.out |   7 +
 .../test/results/clientpositive/llap/bucket4.q.out |   5 +
 .../test/results/clientpositive/llap/bucket5.q.out |  16 +
 .../results/clientpositive/llap/bucket_many.q.out  |   7 +
 .../clientpositive/llap/bucket_map_join_tez2.q.out |   4 +
 .../clientpositive/llap/bucket_num_reducers.q.out  |   3 +
 .../clientpositive/llap/bucket_num_reducers2.q.out |   5 +
 .../clientpositive/llap/bucketmapjoin1.q.out       |  30 +
 .../clientpositive/llap/bucketmapjoin2.q.out       |  27 +
 .../clientpositive/llap/bucketmapjoin3.q.out       |  18 +
 .../clientpositive/llap/bucketmapjoin4.q.out       |  18 +
 .../clientpositive/llap/bucketmapjoin7.q.out       |   8 +
 .../clientpositive/llap/bucketpruning1.q.out       |  52 ++
 .../llap/bucketsortoptimize_insert_2.q.out         |   6 +
 .../llap/bucketsortoptimize_insert_7.q.out         |   3 +
 .../llap/cbo_rp_outer_join_ppr.q.out               |  12 +
 .../clientpositive/llap/cbo_stats_estimation.q.out |   8 +
 .../clientpositive/llap/column_table_stats.q.out   |  16 +
 .../llap/column_table_stats_orc.q.out              |  12 +
 .../clientpositive/llap/columnstats_partlvl.q.out  |   8 +
 .../clientpositive/llap/columnstats_tbllvl.q.out   |   8 +
 .../results/clientpositive/llap/comments.q.out     |   6 +
 .../llap/constantPropagateForSubQuery.q.out        |   6 +
 .../llap/correlationoptimizer12.q.out              |   1 +
 .../llap/correlationoptimizer13.q.out              |   1 +
 .../llap/correlationoptimizer14.q.out              |  64 +-
 .../llap/correlationoptimizer2.q.out               |   6 +
 .../llap/correlationoptimizer6.q.out               |   2 +
 .../llap/correlationoptimizer9.q.out               |   4 +
 .../llap/disable_merge_for_bucketing.q.out         |   5 +
 .../llap/display_colstats_tbllvl.q.out             |   4 +
 .../llap/dynamic_semijoin_reduction.q.out          |  12 +
 .../llap/dynamic_semijoin_reduction_2.q.out        |   1 +
 .../llap/dynamic_semijoin_user_level.q.out         |  12 +
 .../clientpositive/llap/explainanalyze_2.q.out     |  45 +-
 .../clientpositive/llap/explainuser_1.q.out        |  39 +-
 .../clientpositive/llap/explainuser_2.q.out        |  90 ++-
 .../results/clientpositive/llap/filter_aggr.q.out  |   4 +
 .../llap/filter_cond_pushdown2.q.out               |   1 +
 .../llap/filter_join_breaktask.q.out               |  10 +
 .../results/clientpositive/llap/filter_union.q.out |   8 +
 .../clientpositive/llap/join32_lessSize.q.out      |  40 +
 .../clientpositive/llap/list_bucket_dml_10.q.out   |   5 +
 .../clientpositive/llap/llap_nullscan.q.out        |   2 +
 .../results/clientpositive/llap/llap_smb.q.out     |   1 +
 .../clientpositive/llap/mapjoin_mapjoin.q.out      |   6 +
 .../results/clientpositive/llap/mergejoin.q.out    |   6 +
 .../clientpositive/llap/metadataonly1.q.out        |  42 +
 ql/src/test/results/clientpositive/llap/mrr.q.out  |   1 +
 .../llap/murmur_hash_migration.q.out               | 863 +++++++++++++--------
 .../llap/murmur_hash_migration2.q.out              | 336 ++++++++
 .../clientpositive/llap/optimize_nullscan.q.out    |  42 +
 .../clientpositive/llap/ppd_union_view.q.out       |  16 +
 .../results/clientpositive/llap/quotedid_smb.q.out |   1 +
 .../clientpositive/llap/reduce_deduplicate.q.out   |  14 +
 .../test/results/clientpositive/llap/sample1.q.out |   5 +
 .../results/clientpositive/llap/sample10.q.out     |   6 +
 .../results/clientpositive/llap/sharedwork.q.out   |  41 +
 .../results/clientpositive/llap/smb_cache.q.out    |   2 +
 .../clientpositive/llap/smb_mapjoin_14.q.out       |  11 +
 .../clientpositive/llap/smb_mapjoin_15.q.out       |  29 +
 .../clientpositive/llap/smb_mapjoin_4.q.out        |  11 +
 .../clientpositive/llap/smb_mapjoin_5.q.out        |  11 +
 .../clientpositive/llap/smb_mapjoin_6.q.out        |   5 +
 .../test/results/clientpositive/llap/stats11.q.out |  18 +
 .../clientpositive/llap/subquery_in_having.q.out   |   2 +
 .../llap/tez_fixed_bucket_pruning.q.out            |  28 +
 .../llap/tez_join_result_complex.q.out             |   8 +
 .../clientpositive/llap/tez_smb_empty.q.out        |   6 +
 .../results/clientpositive/llap/tez_smb_main.q.out |   2 +
 .../clientpositive/llap/tez_smb_reduce_side.q.out  |   3 +
 .../clientpositive/llap/topnkey_windowing.q.out    |   8 +
 .../clientpositive/llap/unionDistinct_1.q.out      |  53 ++
 .../llap/vector_auto_smb_mapjoin_14.q.out          |  13 +
 .../clientpositive/llap/vectorization_0.q.out      |  16 +
 .../llap/vectorized_bucketmapjoin1.q.out           |   3 +
 .../results/clientpositive/load_dyn_part8.q.out    |  11 +
 .../results/clientpositive/louter_join_ppr.q.out   |  24 +
 ql/src/test/results/clientpositive/merge3.q.out    |  28 +
 .../test/results/clientpositive/mm_buckets.q.out   |   8 +-
 .../offset_limit_global_optimizer.q.out            |  32 +
 .../results/clientpositive/outer_join_ppr.q.out    |  12 +
 .../test/results/clientpositive/parquet_join.q.out |   3 +
 .../clientpositive/parquet_vectorization_0.q.out   |  17 +
 ql/src/test/results/clientpositive/pcr.q.out       | 104 +++
 ql/src/test/results/clientpositive/pcs.q.out       |  28 +
 .../test/results/clientpositive/pointlookup2.q.out |  80 ++
 .../test/results/clientpositive/pointlookup3.q.out |  70 ++
 .../test/results/clientpositive/pointlookup4.q.out |   8 +
 .../results/clientpositive/ppd_join_filter.q.out   |  36 +
 ql/src/test/results/clientpositive/ppd_vc.q.out    |  11 +
 .../clientpositive/ppr_allchildsarenull.q.out      |   4 +
 ql/src/test/results/clientpositive/push_or.q.out   |   4 +
 .../clientpositive/rand_partitionpruner1.q.out     |   2 +
 .../clientpositive/rand_partitionpruner2.q.out     |   7 +
 .../clientpositive/rand_partitionpruner3.q.out     |   4 +
 .../results/clientpositive/regexp_extract.q.out    |  10 +
 .../results/clientpositive/router_join_ppr.q.out   |  24 +
 ql/src/test/results/clientpositive/sample5.q.out   |   7 +
 ql/src/test/results/clientpositive/sample6.q.out   |  35 +
 ql/src/test/results/clientpositive/sample7.q.out   |   7 +
 ql/src/test/results/clientpositive/sample8.q.out   |   6 +
 ql/src/test/results/clientpositive/sample9.q.out   |   2 +
 .../clientpositive/serde_user_properties.q.out     |   6 +
 .../test/results/clientpositive/smb_mapjoin9.q.out |  18 +
 .../results/clientpositive/smb_mapjoin_46.q.out    |  13 +
 .../results/clientpositive/smb_mapjoin_47.q.out    |  22 +
 .../clientpositive/sort_merge_join_desc_1.q.out    |   5 +
 .../clientpositive/sort_merge_join_desc_2.q.out    |   5 +
 .../clientpositive/sort_merge_join_desc_3.q.out    |   5 +
 .../clientpositive/sort_merge_join_desc_5.q.out    |   9 +
 .../clientpositive/sort_merge_join_desc_6.q.out    |   4 +
 .../clientpositive/sort_merge_join_desc_7.q.out    |   4 +
 .../clientpositive/sort_merge_join_desc_8.q.out    |   5 +
 .../spark/auto_join_reordering_values.q.out        |  18 +
 .../spark/auto_sortmerge_join_1.q.out              |  12 +
 .../spark/auto_sortmerge_join_12.q.out             |  14 +
 .../spark/auto_sortmerge_join_2.q.out              |   8 +
 .../spark/auto_sortmerge_join_3.q.out              |  12 +
 .../spark/auto_sortmerge_join_4.q.out              |  12 +
 .../spark/auto_sortmerge_join_5.q.out              |  12 +
 .../spark/auto_sortmerge_join_7.q.out              |  12 +
 .../spark/auto_sortmerge_join_8.q.out              |  12 +
 .../results/clientpositive/spark/bucket2.q.out     |   3 +
 .../results/clientpositive/spark/bucket3.q.out     |   3 +
 .../results/clientpositive/spark/bucket4.q.out     |   3 +
 .../clientpositive/spark/bucket4.q.out_spark       |   2 +
 .../results/clientpositive/spark/bucket5.q.out     |   6 +
 .../clientpositive/spark/bucket_map_join_1.q.out   |   4 +
 .../clientpositive/spark/bucket_map_join_2.q.out   |   4 +
 .../spark/bucket_map_join_spark1.q.out             |   2 +
 .../spark/bucket_map_join_spark2.q.out             |   2 +
 .../spark/bucket_map_join_spark3.q.out             |   2 +
 .../spark/bucket_map_join_spark4.q.out             |   4 +
 .../spark/bucket_map_join_tez2.q.out               |   2 +
 .../clientpositive/spark/bucketmapjoin1.q.out      |  14 +
 .../clientpositive/spark/bucketmapjoin10.q.out     |   4 +
 .../clientpositive/spark/bucketmapjoin11.q.out     |   8 +
 .../clientpositive/spark/bucketmapjoin12.q.out     |   8 +
 .../clientpositive/spark/bucketmapjoin13.q.out     |  16 +
 .../clientpositive/spark/bucketmapjoin2.q.out      |  15 +
 .../clientpositive/spark/bucketmapjoin3.q.out      |  10 +
 .../clientpositive/spark/bucketmapjoin4.q.out      |  10 +
 .../clientpositive/spark/bucketmapjoin5.q.out      |   2 +
 .../clientpositive/spark/bucketmapjoin7.q.out      |   4 +
 .../spark/bucketmapjoin7.q.out_spark               |   2 +
 .../clientpositive/spark/bucketmapjoin8.q.out      |   8 +
 .../clientpositive/spark/bucketmapjoin9.q.out      |   8 +
 .../spark/bucketmapjoin_negative.q.out             |   1 +
 .../spark/bucketmapjoin_negative2.q.out            |   1 +
 .../spark/bucketmapjoin_negative3.q.out            |  18 +
 .../spark/disable_merge_for_bucketing.q.out        |   3 +
 .../spark/disable_merge_for_bucketing.q.out_spark  |   2 +
 .../spark/filter_join_breaktask.q.out              |  10 +
 .../clientpositive/spark/groupby_map_ppr.q.out     |   3 +
 .../spark/groupby_map_ppr_multi_distinct.q.out     |   3 +
 .../results/clientpositive/spark/groupby_ppr.q.out |   3 +
 .../spark/groupby_ppr_multi_distinct.q.out         |   6 +
 .../clientpositive/spark/groupby_sort_1_23.q.out   |  42 +
 .../spark/groupby_sort_skew_1_23.q.out             |  56 ++
 .../spark/infer_bucket_sort_num_buckets.q.out      | 167 +++-
 .../results/clientpositive/spark/input_part2.q.out |   2 +
 .../test/results/clientpositive/spark/join17.q.out |   5 +
 .../test/results/clientpositive/spark/join26.q.out |   1 +
 .../test/results/clientpositive/spark/join32.q.out |   1 +
 .../clientpositive/spark/join32_lessSize.q.out     |  26 +
 .../test/results/clientpositive/spark/join33.q.out |   1 +
 .../test/results/clientpositive/spark/join34.q.out |   7 +
 .../test/results/clientpositive/spark/join35.q.out |  11 +
 .../test/results/clientpositive/spark/join9.q.out  |   5 +
 .../spark/join_filters_overlap.q.out               |  44 ++
 .../clientpositive/spark/join_map_ppr.q.out        |   2 +
 .../clientpositive/spark/list_bucket_dml_10.q.out  |   1 +
 .../clientpositive/spark/list_bucket_dml_2.q.out   |   1 +
 .../clientpositive/spark/load_dyn_part8.q.out      |   6 +
 .../clientpositive/spark/louter_join_ppr.q.out     |  24 +
 .../clientpositive/spark/mapjoin_mapjoin.q.out     |   2 +
 .../clientpositive/spark/optimize_nullscan.q.out   |  42 +
 .../clientpositive/spark/outer_join_ppr.q.out      |  12 +
 .../spark/parquet_vectorization_0.q.out            |  16 +
 ql/src/test/results/clientpositive/spark/pcr.q.out |  80 ++
 .../clientpositive/spark/ppd_join_filter.q.out     |  32 +
 .../clientpositive/spark/reduce_deduplicate.q.out  |   8 +
 .../clientpositive/spark/router_join_ppr.q.out     |  24 +
 .../results/clientpositive/spark/sample1.q.out     |   1 +
 .../results/clientpositive/spark/sample10.q.out    |   6 +
 .../results/clientpositive/spark/sample2.q.out     |   1 +
 .../results/clientpositive/spark/sample4.q.out     |   1 +
 .../results/clientpositive/spark/sample5.q.out     |   1 +
 .../results/clientpositive/spark/sample6.q.out     |  29 +
 .../results/clientpositive/spark/sample7.q.out     |   1 +
 .../results/clientpositive/spark/sample8.q.out     |   6 +
 .../clientpositive/spark/smb_mapjoin_11.q.out      |   3 +
 .../clientpositive/spark/smb_mapjoin_12.q.out      |   6 +
 .../clientpositive/spark/smb_mapjoin_13.q.out      |   8 +
 .../clientpositive/spark/smb_mapjoin_15.q.out      |  20 +
 .../clientpositive/spark/spark_union_merge.q.out   |  16 +
 .../test/results/clientpositive/spark/stats0.q.out |   2 +
 .../clientpositive/spark/transform_ppr1.q.out      |   5 +
 .../clientpositive/spark/transform_ppr2.q.out      |   5 +
 .../results/clientpositive/spark/union22.q.out     |   2 +
 .../results/clientpositive/spark/union24.q.out     |  32 +
 .../results/clientpositive/spark/union_ppr.q.out   |   6 +
 .../clientpositive/spark/vectorization_0.q.out     |  16 +
 ql/src/test/results/clientpositive/stats0.q.out    |  12 +
 .../temp_table_alter_partition_coltype.q.out       |   8 +
 .../temp_table_display_colstats_tbllvl.q.out       |   4 +
 .../temp_table_partition_pruning.q.out             |   6 +
 .../test/results/clientpositive/tez/tez-tag.q.out  |   7 +-
 ql/src/test/results/clientpositive/timestamp.q.out |   6 +
 .../results/clientpositive/transform_ppr1.q.out    |   5 +
 .../results/clientpositive/transform_ppr2.q.out    |   5 +
 .../clientpositive/truncate_column_buckets.q.out   |   8 +-
 .../truncate_column_list_bucket.q.out              |   4 +
 .../test/results/clientpositive/udf_explode.q.out  |   8 +
 .../test/results/clientpositive/udtf_explode.q.out |  14 +
 ql/src/test/results/clientpositive/union22.q.out   |  14 +
 ql/src/test/results/clientpositive/union24.q.out   |  36 +
 ql/src/test/results/clientpositive/union_ppr.q.out |   6 +
 368 files changed, 5956 insertions(+), 701 deletions(-)

diff --git a/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out b/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out
index 82cf417..3783c15 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/insert_into_dynamic_partitions.q.out
@@ -107,8 +107,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: _col1 (type: string), _bucket_number (type: string), _col0 (type: int)
                     null sort order: aaa
+                    numBuckets: 2
                     sort order: +++
                     Map-reduce partition columns: _col1 (type: string)
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -161,6 +163,7 @@ STAGE PLANS:
           expressions: KEY._col0 (type: int), KEY._col1 (type: string), KEY._bucket_number (type: string)
           outputColumnNames: _col0, _col1, _bucket_number
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 1
             directory: ### BLOBSTORE_STAGING_PATH ###
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/insert_into_table.q.out b/itests/hive-blobstore/src/test/results/clientpositive/insert_into_table.q.out
index bab8942..92c785c 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/insert_into_table.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/insert_into_table.q.out
@@ -79,6 +79,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
                     directory: ### BLOBSTORE_STAGING_PATH ###
@@ -123,7 +124,9 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -178,6 +181,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -188,6 +192,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -249,6 +254,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
@@ -372,6 +378,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_directory.q.out b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_directory.q.out
index bda6339..a113a22 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_directory.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_directory.q.out
@@ -90,6 +90,7 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 2 Data size: 180 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
+                bucketingVersion: 2
                 compressed: false
                 GlobalTableId: 1
                 directory: ### BLOBSTORE_STAGING_PATH ###
@@ -100,6 +101,7 @@ STAGE PLANS:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                     properties:
+                      bucketing_version -1
                       columns _col0,_col1
                       columns.types int:string
                       serialization.format 1
@@ -181,6 +183,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: -1
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
@@ -189,6 +192,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1
                     columns.types int:string
                     serialization.format 1
@@ -206,6 +210,7 @@ STAGE PLANS:
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
+              bucketing_version -1
               columns _col0,_col1
               columns.types int:string
               serialization.format 1
@@ -215,6 +220,7 @@ STAGE PLANS:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
               properties:
+                bucketing_version -1
                 columns _col0,_col1
                 columns.types int:string
                 serialization.format 1
@@ -236,6 +242,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: -1
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
@@ -244,6 +251,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1
                     columns.types int:string
                     serialization.format 1
@@ -261,6 +269,7 @@ STAGE PLANS:
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
+              bucketing_version -1
               columns _col0,_col1
               columns.types int:string
               serialization.format 1
@@ -270,6 +279,7 @@ STAGE PLANS:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
               properties:
+                bucketing_version -1
                 columns _col0,_col1
                 columns.types int:string
                 serialization.format 1
@@ -327,6 +337,7 @@ STAGE PLANS:
               outputColumnNames: _col0
               Statistics: Num rows: 2 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
+                bucketingVersion: 2
                 compressed: false
                 GlobalTableId: 1
                 directory: ### BLOBSTORE_STAGING_PATH ###
@@ -337,6 +348,7 @@ STAGE PLANS:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                     properties:
+                      bucketing_version -1
                       columns _col0
                       columns.types int
                       serialization.format 1
@@ -350,6 +362,7 @@ STAGE PLANS:
               outputColumnNames: _col0
               Statistics: Num rows: 2 Data size: 172 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
+                bucketingVersion: 2
                 compressed: false
                 GlobalTableId: 2
                 directory: ### BLOBSTORE_STAGING_PATH ###
@@ -360,6 +373,7 @@ STAGE PLANS:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                     properties:
+                      bucketing_version -1
                       columns _col0
                       columns.types string
                       serialization.format 1
@@ -441,6 +455,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: -1
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
@@ -449,6 +464,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0
                     columns.types int
                     serialization.format 1
@@ -466,6 +482,7 @@ STAGE PLANS:
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
+              bucketing_version -1
               columns _col0
               columns.types int
               serialization.format 1
@@ -475,6 +492,7 @@ STAGE PLANS:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
               properties:
+                bucketing_version -1
                 columns _col0
                 columns.types int
                 serialization.format 1
@@ -496,6 +514,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: -1
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
@@ -504,6 +523,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0
                     columns.types int
                     serialization.format 1
@@ -521,6 +541,7 @@ STAGE PLANS:
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
+              bucketing_version -1
               columns _col0
               columns.types int
               serialization.format 1
@@ -530,6 +551,7 @@ STAGE PLANS:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
               properties:
+                bucketing_version -1
                 columns _col0
                 columns.types int
                 serialization.format 1
@@ -561,6 +583,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: -1
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
@@ -569,6 +592,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0
                     columns.types string
                     serialization.format 1
@@ -586,6 +610,7 @@ STAGE PLANS:
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
+              bucketing_version -1
               columns _col0
               columns.types string
               serialization.format 1
@@ -595,6 +620,7 @@ STAGE PLANS:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
               properties:
+                bucketing_version -1
                 columns _col0
                 columns.types string
                 serialization.format 1
@@ -616,6 +642,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: -1
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
@@ -624,6 +651,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0
                     columns.types string
                     serialization.format 1
@@ -641,6 +669,7 @@ STAGE PLANS:
             input format: org.apache.hadoop.mapred.TextInputFormat
             output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
             properties:
+              bucketing_version -1
               columns _col0
               columns.types string
               serialization.format 1
@@ -650,6 +679,7 @@ STAGE PLANS:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
               properties:
+                bucketing_version -1
                 columns _col0
                 columns.types string
                 serialization.format 1
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out
index fc8f3d0..91e95c4 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_dynamic_partitions.q.out
@@ -125,8 +125,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: _col1 (type: string), _bucket_number (type: string), _col0 (type: int)
                     null sort order: aaa
+                    numBuckets: 2
                     sort order: +++
                     Map-reduce partition columns: _col1 (type: string)
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -179,6 +181,7 @@ STAGE PLANS:
           expressions: KEY._col0 (type: int), KEY._col1 (type: string), KEY._bucket_number (type: string)
           outputColumnNames: _col0, _col1, _bucket_number
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 1
             directory: ### BLOBSTORE_STAGING_PATH ###
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out
index 9903d69..96e77ed 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/insert_overwrite_table.q.out
@@ -87,6 +87,7 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
                     directory: ### BLOBSTORE_STAGING_PATH ###
@@ -131,7 +132,9 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 424 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -186,6 +189,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -196,6 +200,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -257,6 +262,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
@@ -380,6 +386,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
               directory: ### BLOBSTORE_STAGING_PATH ###
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/write_final_output_blobstore.q.out b/itests/hive-blobstore/src/test/results/clientpositive/write_final_output_blobstore.q.out
index 2addf92..2dd98ef 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/write_final_output_blobstore.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/write_final_output_blobstore.q.out
@@ -61,8 +61,10 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
@@ -130,6 +132,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -154,8 +157,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -195,6 +200,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 1
             directory: ### BLOBSTORE_STAGING_PATH ###
@@ -242,6 +248,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 428 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -252,6 +259,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       properties:
+                        bucketing_version -1
                         columns _col0
                         columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                         escape.delim \
@@ -341,8 +349,10 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
@@ -410,6 +420,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -434,8 +445,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -475,6 +488,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 1
             directory: ### BLOBSTORE_STAGING_PATH ###
@@ -522,6 +536,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 428 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -532,6 +547,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       properties:
+                        bucketing_version -1
                         columns _col0
                         columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                         escape.delim \
diff --git a/itests/src/test/resources/testconfiguration.properties b/itests/src/test/resources/testconfiguration.properties
index 5468728..917b717 100644
--- a/itests/src/test/resources/testconfiguration.properties
+++ b/itests/src/test/resources/testconfiguration.properties
@@ -679,6 +679,7 @@ minillaplocal.query.files=\
   multiMapJoin2.q,\
   multi_in_clause.q,\
   murmur_hash_migration.q,\
+  murmur_hash_migration2.q,\
   non_native_window_udf.q,\
   optimize_join_ptp.q,\
   orc_analyze.q,\
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java
index 3e1100c..babbffd 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/JoinUtil.java
@@ -21,8 +21,10 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Properties;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.persistence.RowContainer;
 import org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -215,7 +217,7 @@ public class JoinUtil {
 
     // Compute the values
     int reserve = hasFilter ? valueFields.size() + 1 : valueFields.size();
-    List<Object> nr = new ArrayList<Object>(reserve);   
+    List<Object> nr = new ArrayList<Object>(reserve);
     for (int i = 0; i < valueFields.size(); i++) {
       nr.add(ObjectInspectorUtils.copyToStandardObject(valueFields.get(i)
           .evaluate(row), valueFieldsOI.get(i),
@@ -343,16 +345,16 @@ public class JoinUtil {
       // remove the last ','
       colNames.setLength(colNames.length() - 1);
       colTypes.setLength(colTypes.length() - 1);
+      Properties props = new Properties();
+      props.put(org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.ctrlaCode);
+      props.put(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS, colNames.toString());
+      props.put(org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES, colTypes.toString());
+      props.put(serdeConstants.SERIALIZATION_LIB, LazyBinarySerDe.class.getName());
+      props.put(hive_metastoreConstants.TABLE_BUCKETING_VERSION, "-1");
       TableDesc tblDesc = new TableDesc(
-          SequenceFileInputFormat.class, HiveSequenceFileOutputFormat.class,
-          Utilities.makeProperties(
-          org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, ""
-          + Utilities.ctrlaCode,
-          org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMNS, colNames
-          .toString(),
-          org.apache.hadoop.hive.serde.serdeConstants.LIST_COLUMN_TYPES,
-          colTypes.toString(),
-          serdeConstants.SERIALIZATION_LIB,LazyBinarySerDe.class.getName()));
+          SequenceFileInputFormat.class,
+          HiveSequenceFileOutputFormat.class,
+          props);
       spillTableDesc[tag] = tblDesc;
     }
     return spillTableDesc;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
index 753f25b..a11cabf 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Operator.java
@@ -92,7 +92,6 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
   protected final transient Collection<Future<?>> asyncInitOperations = new HashSet<>();
   private String marker;
 
-  protected int bucketingVersion = -1;
   // It can be optimized later so that an operator operator (init/close) is performed
   // only after that operation has been performed on all the parents. This will require
   // initializing the whole tree in all the mappers (which might be required for mappers
@@ -1544,12 +1543,4 @@ public abstract class Operator<T extends OperatorDesc> implements Serializable,C
     }
     return true;
   }
-
-  public void setBucketingVersion(int bucketingVersion) {
-    this.bucketingVersion = bucketingVersion;
-  }
-
-  public int getBucketingVersion() {
-    return bucketingVersion;
-  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
index e97fcef..da26e4f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorFactory.java
@@ -40,13 +40,10 @@ import org.apache.hadoop.hive.ql.exec.vector.VectorSparkHashTableSinkOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorSparkPartitionPruningSinkOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorTopNKeyOperator;
 import org.apache.hadoop.hive.ql.exec.vector.VectorizationContext;
-import org.apache.hadoop.hive.ql.exec.vector.reducesink.VectorReduceSinkCommonOperator;
 import org.apache.hadoop.hive.ql.exec.vector.ptf.VectorPTFOperator;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.optimizer.spark.SparkPartitionPruningSinkDesc;
 import org.apache.hadoop.hive.ql.parse.spark.SparkPartitionPruningSinkOperator;
-import org.apache.hadoop.hive.ql.plan.AbstractOperatorDesc;
-import org.apache.hadoop.hive.ql.plan.AbstractVectorDesc;
 import org.apache.hadoop.hive.ql.plan.AppMasterEventDesc;
 import org.apache.hadoop.hive.ql.plan.CollectDesc;
 import org.apache.hadoop.hive.ql.plan.CommonMergeJoinDesc;
@@ -266,9 +263,6 @@ public final class OperatorFactory {
     Operator<T> ret = get(oplist0.getCompilationOpContext(), (Class<T>) conf.getClass());
     ret.setConf(conf);
 
-    // Set the bucketing Version
-    ret.setBucketingVersion(oplist0.getBucketingVersion());
-
     // Add the new operator as child of each of the passed in operators
     List<Operator> children = oplist0.getChildOperators();
     children.add(ret);
@@ -340,7 +334,9 @@ public final class OperatorFactory {
     Operator<T> ret = get(ctx, (Class<T>) conf.getClass());
     ret.setConf(conf);
     ret.setSchema(rwsch);
-    if (oplist.length == 0) return ret;
+    if (oplist.length == 0) {
+      return ret;
+    }
 
     // Add the new operator as child of each of the passed in operators
     for (Operator op : oplist) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
index ce0f08d..964c98d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
@@ -233,7 +233,7 @@ public class ReduceSinkOperator extends TerminalOperator<ReduceSinkDesc>
       // incase of ACID updates/deletes.
       boolean acidOp = conf.getWriteType() == AcidUtils.Operation.UPDATE ||
           conf.getWriteType() == AcidUtils.Operation.DELETE;
-      hashFunc = bucketingVersion == 2 && !acidOp ?
+      hashFunc = getConf().getBucketingVersion() == 2 && !acidOp ?
           ObjectInspectorUtils::getBucketHashCode :
           ObjectInspectorUtils::getBucketHashCodeOld;
     } catch (Exception e) {
@@ -430,7 +430,7 @@ public class ReduceSinkOperator extends TerminalOperator<ReduceSinkDesc>
    * For Acid Update/Delete case, we expect a single partitionEval of the form
    * UDFToInteger(ROW__ID) and buckNum == -1 so that the result of this method
    * is to return the bucketId extracted from ROW__ID unless it optimized by
-   * {@link org.apache.hadoop.hive.ql.optimizer.SortedDynPartitionOptimizer} 
+   * {@link org.apache.hadoop.hive.ql.optimizer.SortedDynPartitionOptimizer}
    */
   private int computeHashCode(Object row, int buckNum) throws HiveException {
     // Evaluate the HashCode
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
index 535c245..5d244ec 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
@@ -795,12 +795,14 @@ public final class Utilities {
 
   // column names and column types are all delimited by comma
   public static TableDesc getTableDesc(String cols, String colTypes) {
+    Properties properties = new Properties();
+    properties.put(serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.ctrlaCode);
+    properties.put(serdeConstants.LIST_COLUMNS, cols);
+    properties.put(serdeConstants.LIST_COLUMN_TYPES, colTypes);
+    properties.put(serdeConstants.SERIALIZATION_LIB, LazySimpleSerDe.class.getName());
+    properties.put(hive_metastoreConstants.TABLE_BUCKETING_VERSION, "-1");
     return (new TableDesc(SequenceFileInputFormat.class,
-        HiveSequenceFileOutputFormat.class, Utilities.makeProperties(
-        serdeConstants.SERIALIZATION_FORMAT, "" + Utilities.ctrlaCode,
-        serdeConstants.LIST_COLUMNS, cols,
-        serdeConstants.LIST_COLUMN_TYPES, colTypes,
-        serdeConstants.SERIALIZATION_LIB,LazySimpleSerDe.class.getName())));
+        HiveSequenceFileOutputFormat.class, properties));
   }
 
   public static PartitionDesc getPartitionDesc(Partition part, TableDesc tableDesc) throws
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/PTFRowContainer.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/PTFRowContainer.java
index ca5f585..156f84d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/PTFRowContainer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/persistence/PTFRowContainer.java
@@ -29,6 +29,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.exec.FileSinkOperator.RecordWriter;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -237,6 +238,7 @@ public class PTFRowContainer<Row extends List<Object>> extends RowContainer<Row>
         serdeConstants.SERIALIZATION_FORMAT, ""+ Utilities.ctrlaCode,
         serdeConstants.LIST_COLUMNS, colNames.toString(),
         serdeConstants.LIST_COLUMN_TYPES,colTypes.toString(),
+        hive_metastoreConstants.TABLE_BUCKETING_VERSION, "-1",
         serdeConstants.SERIALIZATION_LIB,LazyBinarySerDe.class.getName()));
     return tblDesc;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java
index 2192274..bf86b48 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/reducesink/VectorReduceSinkObjectHashOperator.java
@@ -184,7 +184,7 @@ public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOp
     }
 
     // Set hashFunc
-    hashFunc = bucketingVersion == 2 && !vectorDesc.getIsAcidChange() ?
+    hashFunc = getConf().getBucketingVersion() == 2 && !vectorDesc.getIsAcidChange() ?
       ObjectInspectorUtils::getBucketHashCode :
       ObjectInspectorUtils::getBucketHashCodeOld;
 
@@ -232,21 +232,21 @@ public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOp
           ve.evaluate(batch);
         }
       }
-  
+
       // Perform any value expressions.  Results will go into scratch columns.
       if (reduceSinkValueExpressions != null) {
         for (VectorExpression ve : reduceSinkValueExpressions) {
           ve.evaluate(batch);
         }
       }
-  
+
       // Perform any bucket expressions.  Results will go into scratch columns.
       if (reduceSinkBucketExpressions != null) {
         for (VectorExpression ve : reduceSinkBucketExpressions) {
           ve.evaluate(batch);
         }
       }
-  
+
       // Perform any partition expressions.  Results will go into scratch columns.
       if (reduceSinkPartitionExpressions != null) {
         for (VectorExpression ve : reduceSinkPartitionExpressions) {
@@ -296,7 +296,9 @@ public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOp
 
   private void processKey(VectorizedRowBatch batch, int batchIndex, int tag)
   throws HiveException{
-    if (isEmptyKey) return;
+    if (isEmptyKey) {
+      return;
+    }
 
     try {
       keyBinarySortableSerializeWrite.reset();
@@ -318,7 +320,9 @@ public class VectorReduceSinkObjectHashOperator extends VectorReduceSinkCommonOp
   }
 
   private void processValue(VectorizedRowBatch batch, int batchIndex)  throws HiveException {
-    if (isEmptyValue) return;
+    if (isEmptyValue) {
+      return;
+    }
 
     try {
       valueLazyBinarySerializeWrite.reset();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
index 3f30c8c..8c4496a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/AbstractSMBJoinProc.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.ql.exec.DummyStoreOperator;
 import org.apache.hadoop.hive.ql.exec.JoinOperator;
 import org.apache.hadoop.hive.ql.exec.MapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.OperatorFactory;
 import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
 import org.apache.hadoop.hive.ql.exec.SMBMapJoinOperator;
 import org.apache.hadoop.hive.ql.exec.TableScanOperator;
@@ -47,6 +48,7 @@ import org.apache.hadoop.hive.ql.parse.PrunedPartitionList;
 import org.apache.hadoop.hive.ql.parse.QB;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.parse.TableAccessAnalyzer;
+import org.apache.hadoop.hive.ql.plan.DummyStoreDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
 import org.apache.hadoop.hive.ql.plan.JoinCondDesc;
 import org.apache.hadoop.hive.ql.plan.JoinDesc;
@@ -185,7 +187,9 @@ abstract public class AbstractSMBJoinProc extends AbstractBucketJoinProc impleme
         par.getChildOperators().add(index, smbJop);
       }
       else {
-        DummyStoreOperator dummyStoreOp = new DummyStoreOperator(par.getCompilationOpContext());
+        DummyStoreOperator dummyStoreOp =
+            (DummyStoreOperator) OperatorFactory.get(par.getCompilationOpContext(), new DummyStoreDesc());
+
         par.getChildOperators().add(index, dummyStoreOp);
 
         List<Operator<? extends OperatorDesc>> childrenOps =
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketVersionPopulator.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketVersionPopulator.java
new file mode 100644
index 0000000..421e4e1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/BucketVersionPopulator.java
@@ -0,0 +1,229 @@
+/*
+ * 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.hadoop.hive.ql.optimizer;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.Stack;
+
+import org.apache.hadoop.hive.ql.exec.FileSinkOperator;
+import org.apache.hadoop.hive.ql.exec.Operator;
+import org.apache.hadoop.hive.ql.exec.ReduceSinkOperator;
+import org.apache.hadoop.hive.ql.exec.TableScanOperator;
+import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
+import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
+import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lib.NodeProcessorCtx;
+import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
+import org.apache.hadoop.hive.ql.lib.SemanticGraphWalker;
+import org.apache.hadoop.hive.ql.lib.SemanticNodeProcessor;
+import org.apache.hadoop.hive.ql.lib.SemanticRule;
+import org.apache.hadoop.hive.ql.parse.ParseContext;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Sets;
+
+/**
+ * This class analyzes and sets the bucketing versions.
+ *
+ * A set of data values can be distributed into N buckets differently depending on the used hashing algorithm.
+ * Hive right now supports multiple hashing algorithms - the actual algo is identified by "bucketingVersion".
+ *
+ * Bucketing version can be re-select after every Reduce Sink; because a full shuffle can re-distribute the data according to a new hash algo as well.
+ *
+ * Depending on the table Hive might need to write it's data in some specific bucketing version.
+ *
+ * In case a bucketed table is read from the table location; the data should be threated as described by the table's bucketing_version property.
+ *
+ */
+public class BucketVersionPopulator extends Transform {
+
+  protected static final Logger LOG = LoggerFactory.getLogger(BucketVersionPopulator.class);
+
+  @Override
+  public ParseContext transform(ParseContext pctx) throws SemanticException {
+    Set<OpGroup> groups = findOpGroups(pctx);
+    assignGroupVersions(groups);
+    return pctx;
+  }
+
+  private void assignGroupVersions(Set<OpGroup> groups) {
+    for (OpGroup opGroup : groups) {
+      opGroup.analyzeBucketVersion();
+      opGroup.setBucketVersion();
+    }
+
+  }
+
+  static class BucketVersionProcessorCtx implements NodeProcessorCtx {
+    Set<OpGroup> groups = new HashSet<OpGroup>();
+  }
+
+  private Set<OpGroup> findOpGroups(ParseContext pctx) throws SemanticException {
+
+    BucketVersionProcessorCtx ctx = new BucketVersionProcessorCtx();
+
+    Map<SemanticRule, SemanticNodeProcessor> opRules = new LinkedHashMap<SemanticRule, SemanticNodeProcessor>();
+
+    SemanticDispatcher disp = new DefaultRuleDispatcher(new IdentifyBucketGroups(), opRules, ctx);
+    SemanticGraphWalker ogw = new DefaultGraphWalker(disp);
+
+    ArrayList<Node> topNodes = new ArrayList<Node>();
+    topNodes.addAll(pctx.getTopOps().values());
+    ogw.startWalking(topNodes, null);
+    return ctx.groups;
+  }
+
+  /**
+   * This rule decomposes the operator tree into group which may have different bucketing versions.
+   */
+  private static class IdentifyBucketGroups implements SemanticNodeProcessor {
+
+    @Override
+    public Object process(Node nd, Stack<Node> stack, NodeProcessorCtx procCtx, Object... nodeOutputs)
+        throws SemanticException {
+      Operator<?> o = (Operator<?>) nd;
+      OpGroup g;
+      if (nodeOutputs.length == 0) {
+        g = newGroup(procCtx);
+      } else {
+        g = (OpGroup) nodeOutputs[0];
+      }
+      for (int i = 1; i < nodeOutputs.length; i++) {
+        g.merge((OpGroup) nodeOutputs[i]);
+      }
+      g.add(o);
+      if (o instanceof ReduceSinkOperator) {
+        // start a new group before the reduceSinkOperator
+        return newGroup(procCtx);
+      } else {
+        return g;
+      }
+    }
+
+    private OpGroup newGroup(NodeProcessorCtx procCtx) {
+      BucketVersionProcessorCtx ctx = (BucketVersionProcessorCtx) procCtx;
+      OpGroup g = new OpGroup();
+      ctx.groups.add(g);
+      return g;
+    }
+  }
+
+  /**
+   * This class represents the version required by an Operator.
+   */
+  private static class OperatorBucketingVersionInfo {
+
+    private Operator<?> op;
+    private int bucketingVersion;
+
+    public OperatorBucketingVersionInfo(Operator<?> op, int bucketingVersion) {
+      this.op = op;
+      this.bucketingVersion = bucketingVersion;
+    }
+
+    @Override
+    public String toString() {
+      return String.format("[op: %s, bucketingVersion=%d]", op, bucketingVersion);
+    }
+  }
+
+  /**
+   * A Group of operators which must have the same bucketing version.
+   */
+  private static class OpGroup {
+    Set<Operator<?>> members = Sets.newIdentityHashSet();
+    int version = -1;
+
+    public OpGroup() {
+    }
+
+    public void add(Operator<?> o) {
+      members.add(o);
+    }
+
+    public void setBucketVersion() {
+      for (Operator<?> operator : members) {
+        operator.getConf().setBucketingVersion(version);
+        LOG.debug("Bucketing version for {} is set to {}", operator, version);
+      }
+    }
+
+    List<OperatorBucketingVersionInfo> getBucketingVersions() {
+      List<OperatorBucketingVersionInfo> ret = new ArrayList<>();
+      for (Operator<?> operator : members) {
+        if (operator instanceof TableScanOperator) {
+          TableScanOperator tso = (TableScanOperator) operator;
+          int bucketingVersion = tso.getConf().getTableMetadata().getBucketingVersion();
+          int numBuckets = tso.getConf().getNumBuckets();
+          if (numBuckets > 1) {
+            ret.add(new OperatorBucketingVersionInfo(operator, bucketingVersion));
+          } else {
+            LOG.info("not considering bucketingVersion for: %s because it has %d<2 buckets ", tso, numBuckets);
+          }
+        }
+        if (operator instanceof FileSinkOperator) {
+          FileSinkOperator fso = (FileSinkOperator) operator;
+          int bucketingVersion = fso.getConf().getTableInfo().getBucketingVersion();
+          ret.add(new OperatorBucketingVersionInfo(operator, bucketingVersion));
+        }
+      }
+      return ret;
+    }
+
+    public void analyzeBucketVersion() {
+      List<OperatorBucketingVersionInfo> bucketingVersions = getBucketingVersions();
+      try {
+        for (OperatorBucketingVersionInfo info : bucketingVersions) {
+          setVersion(info.bucketingVersion);
+        }
+      } catch (Exception e) {
+        throw new RuntimeException("Error setting bucketingVersion for group: " + bucketingVersions, e);
+      }
+      if (version == -1) {
+        // use version 2 if possible
+        version = 2;
+      }
+    }
+
+    private void setVersion(int newVersion) {
+      if (version == newVersion || newVersion == -1) {
+        return;
+      }
+      if (version == -1) {
+        version = newVersion;
+        return;
+      }
+      throw new RuntimeException("Unable to set version");
+    }
+
+    public void merge(OpGroup opGroup) {
+      for (Operator<?> operator : opGroup.members) {
+        add(operator);
+      }
+      opGroup.members.clear();
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
index 3207f39..655b5f1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/ConvertJoinMapJoin.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hive.ql.parse.ParseContext;
 import org.apache.hadoop.hive.ql.parse.SemanticException;
 import org.apache.hadoop.hive.ql.plan.ColStatistics;
 import org.apache.hadoop.hive.ql.plan.CommonMergeJoinDesc;
+import org.apache.hadoop.hive.ql.plan.DummyStoreDesc;
 import org.apache.hadoop.hive.ql.plan.DynamicPruningEventDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeColumnDesc;
 import org.apache.hadoop.hive.ql.plan.ExprNodeDesc;
@@ -207,7 +208,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
     // map join operator by default has no bucket cols and num of reduce sinks
     // reduced by 1
     mapJoinOp.setOpTraits(new OpTraits(null, -1, null,
-        joinOp.getOpTraits().getNumReduceSinks(), joinOp.getOpTraits().getBucketingVersion()));
+        joinOp.getOpTraits().getNumReduceSinks()));
     preserveOperatorInfos(mapJoinOp, joinOp, context);
     // propagate this change till the next RS
     for (Operator<? extends OperatorDesc> childOp : mapJoinOp.getChildOperators()) {
@@ -543,9 +544,9 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
     context.parseContext.getContext().getPlanMapper().link(joinOp, mergeJoinOp);
     int numReduceSinks = joinOp.getOpTraits().getNumReduceSinks();
     OpTraits opTraits = new OpTraits(joinOp.getOpTraits().getBucketColNames(), numBuckets,
-      joinOp.getOpTraits().getSortCols(), numReduceSinks,
-      joinOp.getOpTraits().getBucketingVersion());
+        joinOp.getOpTraits().getSortCols(), numReduceSinks);
     mergeJoinOp.setOpTraits(opTraits);
+    mergeJoinOp.getConf().setBucketingVersion(joinOp.getConf().getBucketingVersion());
     preserveOperatorInfos(mergeJoinOp, joinOp, context);
 
     for (Operator<? extends OperatorDesc> parentOp : joinOp.getParentOperators()) {
@@ -590,8 +591,8 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
         }
 
         // insert the dummy store operator here
-        DummyStoreOperator dummyStoreOp = new TezDummyStoreOperator(
-            mergeJoinOp.getCompilationOpContext());
+        DummyStoreOperator dummyStoreOp = new TezDummyStoreOperator(mergeJoinOp.getCompilationOpContext());
+        dummyStoreOp.setConf(new DummyStoreDesc());
         dummyStoreOp.setParentOperators(new ArrayList<Operator<? extends OperatorDesc>>());
         dummyStoreOp.setChildOperators(new ArrayList<Operator<? extends OperatorDesc>>());
         dummyStoreOp.getChildOperators().add(mergeJoinOp);
@@ -611,8 +612,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
       return;
     }
     currentOp.setOpTraits(new OpTraits(opTraits.getBucketColNames(),
-      opTraits.getNumBuckets(), opTraits.getSortCols(), opTraits.getNumReduceSinks(),
-            opTraits.getBucketingVersion()));
+        opTraits.getNumBuckets(), opTraits.getSortCols(), opTraits.getNumReduceSinks()));
     for (Operator<? extends OperatorDesc> childOp : currentOp.getChildOperators()) {
       if ((childOp instanceof ReduceSinkOperator) || (childOp instanceof GroupByOperator)) {
         break;
@@ -670,8 +670,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
 
     // we can set the traits for this join operator
     opTraits = new OpTraits(joinOp.getOpTraits().getBucketColNames(),
-        tezBucketJoinProcCtx.getNumBuckets(), null, joinOp.getOpTraits().getNumReduceSinks(),
-        joinOp.getOpTraits().getBucketingVersion());
+        tezBucketJoinProcCtx.getNumBuckets(), null, joinOp.getOpTraits().getNumReduceSinks());
     mapJoinOp.setOpTraits(opTraits);
     preserveOperatorInfos(mapJoinOp, joinOp, context);
     setNumberOfBucketsOnChildren(mapJoinOp);
@@ -821,10 +820,9 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
     for (Operator<? extends OperatorDesc> parentOp : joinOp.getParentOperators()) {
       // Check if the parent is coming from a table scan, if so, what is the version of it.
       assert parentOp.getParentOperators() != null && parentOp.getParentOperators().size() == 1;
-      Operator<?> op = parentOp.getParentOperators().get(0);
-      while(op != null && !(op instanceof TableScanOperator
-              || op instanceof ReduceSinkOperator
-              || op instanceof CommonJoinOperator)) {
+      Operator<?> op = parentOp;
+      while (op != null && !(op instanceof TableScanOperator || op instanceof ReduceSinkOperator
+          || op instanceof CommonJoinOperator)) {
         // If op has parents it is guaranteed to be 1.
         List<Operator<?>> parents = op.getParentOperators();
         Preconditions.checkState(parents.size() == 0 || parents.size() == 1);
@@ -832,8 +830,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
       }
 
       if (op instanceof TableScanOperator) {
-        int localVersion = ((TableScanOperator)op).getConf().
-                getTableMetadata().getBucketingVersion();
+        int localVersion = ((TableScanOperator) op).getConf().getTableMetadata().getBucketingVersion();
         if (bucketingVersion == -1) {
           bucketingVersion = localVersion;
         } else if (bucketingVersion != localVersion) {
@@ -1543,8 +1540,7 @@ public class ConvertJoinMapJoin implements SemanticNodeProcessor {
             joinOp.getOpTraits().getBucketColNames(),
             numReducers,
             null,
-            joinOp.getOpTraits().getNumReduceSinks(),
-            joinOp.getOpTraits().getBucketingVersion());
+            joinOp.getOpTraits().getNumReduceSinks());
         mapJoinOp.setOpTraits(opTraits);
         preserveOperatorInfos(mapJoinOp, joinOp, context);
         // propagate this change till the next RS
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
index 07bd6e7..70bc13e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/Optimizer.java
@@ -191,6 +191,8 @@ public class Optimizer {
       transformations.add(new FixedBucketPruningOptimizer(compatMode));
     }
 
+    transformations.add(new BucketVersionPopulator());
+
     if(HiveConf.getBoolVar(hiveConf, HiveConf.ConfVars.HIVEOPTREDUCEDEDUPLICATION) &&
         !isTezExecEngine) {
       transformations.add(new ReduceSinkDeDuplication());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
index c98417a..21f6e21 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionOptimizer.java
@@ -45,7 +45,6 @@ import org.apache.hadoop.hive.ql.exec.SelectOperator;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.Utilities.ReduceField;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
-import org.apache.hadoop.hive.ql.io.RecordIdentifier;
 import org.apache.hadoop.hive.ql.lib.DefaultGraphWalker;
 import org.apache.hadoop.hive.ql.lib.DefaultRuleDispatcher;
 import org.apache.hadoop.hive.ql.lib.SemanticDispatcher;
@@ -222,7 +221,7 @@ public class SortedDynPartitionOptimizer extends Transform {
         /**
          * ROW__ID is always the 1st column of Insert representing Update/Delete operation
          * (set up in {@link org.apache.hadoop.hive.ql.parse.UpdateDeleteSemanticAnalyzer})
-         * and we wrap it in UDFToInteger 
+         * and we wrap it in UDFToInteger
          * (in {@link org.apache.hadoop.hive.ql.parse.SemanticAnalyzer#getPartitionColsFromBucketColsForUpdateDelete(Operator, boolean)})
          * which extracts bucketId from it
          * see {@link org.apache.hadoop.hive.ql.udf.UDFToInteger#evaluate(RecordIdentifier)}*/
@@ -285,6 +284,7 @@ public class SortedDynPartitionOptimizer extends Transform {
       // Create ReduceSink operator
       ReduceSinkOperator rsOp = getReduceSinkOp(partitionPositions, sortPositions, sortOrder, sortNullOrder,
         allRSCols, bucketColumns, numBuckets, fsParent, fsOp.getConf().getWriteType());
+      rsOp.getConf().setBucketingVersion(fsOp.getConf().getBucketingVersion());
 
       List<ExprNodeDesc> descs = new ArrayList<ExprNodeDesc>(allRSCols.size());
       List<String> colNames = new ArrayList<String>();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
index d458ebb..9d80f08 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/SortedDynPartitionTimeGranularityOptimizer.java
@@ -211,6 +211,7 @@ public class SortedDynPartitionTimeGranularityOptimizer extends Transform {
       }
       ReduceSinkOperator rsOp = getReduceSinkOp(keyPositions, sortOrder,
           sortNullOrder, allRSCols, granularitySelOp, fsOp.getConf().getWriteType());
+      rsOp.getConf().setBucketingVersion(fsOp.getConf().getBucketingVersion());
 
       // Create backtrack SelectOp
       final List<ExprNodeDesc> descs = new ArrayList<>(allRSCols.size());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveTableScanVisitor.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveTableScanVisitor.java
index 72411ec..14958aa 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveTableScanVisitor.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/translator/opconventer/HiveTableScanVisitor.java
@@ -108,7 +108,6 @@ class HiveTableScanVisitor extends HiveRelNodeVisitor<HiveTableScan> {
     // 2. Setup TableScan
     TableScanOperator ts = (TableScanOperator) OperatorFactory.get(
         hiveOpConverter.getSemanticAnalyzer().getOpContext(), tsd, new RowSchema(colInfos));
-    ts.setBucketingVersion(tsd.getTableMetadata().getBucketingVersion());
 
     //now that we let Calcite process subqueries we might have more than one
     // tablescan with same alias.
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplicationUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplicationUtils.java
index 28ddecc..4681ba7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplicationUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/correlation/ReduceSinkDeDuplicationUtils.java
@@ -208,7 +208,7 @@ public class ReduceSinkDeDuplicationUtils {
           throws SemanticException {
     return strictMerge(cRS, ImmutableList.of(pRS));
   }
-  
+
   public static boolean strictMerge(ReduceSinkOperator cRS, List<ReduceSinkOperator> pRSs)
           throws SemanticException {
     ReduceSinkDesc cRSc = cRS.getConf();
@@ -226,7 +226,7 @@ public class ReduceSinkDeDuplicationUtils {
       if (moveRSOrderTo == null) {
         return false;
       }
-  
+
       int cKeySize = cRSc.getKeyCols().size();
       for (int i = 0; i < cKeySize; i++) {
         ExprNodeDesc cExpr = cRSc.getKeyCols().get(i);
@@ -240,7 +240,7 @@ public class ReduceSinkDeDuplicationUtils {
           return false;
         }
       }
-  
+
       int cPartSize = cRSc.getPartitionCols().size();
       for (int i = 0; i < cPartSize; i++) {
         ExprNodeDesc cExpr = cRSc.getPartitionCols().get(i);
@@ -309,6 +309,9 @@ public class ReduceSinkDeDuplicationUtils {
     if (cConf.getDistinctColumnIndices().size() >= 2) {
       return null;
     }
+    if (cConf.getBucketingVersion() != pConf.getBucketingVersion()) {
+      return null;
+    }
     Integer moveReducerNumTo = checkNumReducer(cConf.getNumReducers(), pConf.getNumReducers());
     if (moveReducerNumTo == null ||
         moveReducerNumTo > 0 && cConf.getNumReducers() < minReducer) {
@@ -480,6 +483,9 @@ public class ReduceSinkDeDuplicationUtils {
   // ensure SEL does not branch
   protected static boolean checkSelectSingleBranchOnly(ReduceSinkOperator cRS, ReduceSinkOperator pRS) {
     Operator<? extends OperatorDesc> parent = cRS.getParentOperators().get(0);
+    if (cRS.getConf().getBucketingVersion() != pRS.getConf().getBucketingVersion()) {
+      return false;
+    }
     while (parent != pRS) {
       assert parent.getNumParent() == 1;
       if (!(parent instanceof SelectOperator)) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/OpTraitsRulesProcFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/OpTraitsRulesProcFactory.java
index c935b74..5692eb7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/OpTraitsRulesProcFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/metainfo/annotation/OpTraitsRulesProcFactory.java
@@ -97,12 +97,10 @@ public class OpTraitsRulesProcFactory {
       List<List<String>> listBucketCols = new ArrayList<List<String>>();
       int numBuckets = -1;
       int numReduceSinks = 1;
-      int bucketingVersion = -1;
       OpTraits parentOpTraits = rs.getParentOperators().get(0).getOpTraits();
       if (parentOpTraits != null) {
         numBuckets = parentOpTraits.getNumBuckets();
         numReduceSinks += parentOpTraits.getNumReduceSinks();
-        bucketingVersion = parentOpTraits.getBucketingVersion();
       }
 
       List<String> bucketCols = new ArrayList<>();
@@ -162,9 +160,8 @@ public class OpTraitsRulesProcFactory {
 
       listBucketCols.add(bucketCols);
       OpTraits opTraits = new OpTraits(listBucketCols, numBuckets,
-              listBucketCols, numReduceSinks, bucketingVersion);
+          listBucketCols, numReduceSinks);
       rs.setOpTraits(opTraits);
-      rs.setBucketingVersion(bucketingVersion);
       return null;
     }
   }
@@ -243,7 +240,7 @@ public class OpTraitsRulesProcFactory {
       }
       // num reduce sinks hardcoded to 0 because TS has no parents
       OpTraits opTraits = new OpTraits(bucketColsList, numBuckets,
-              sortedColsList, 0, table.getBucketingVersion());
+          sortedColsList, 0);
       ts.setOpTraits(opTraits);
       return null;
     }
@@ -269,15 +266,13 @@ public class OpTraitsRulesProcFactory {
 
       List<List<String>> listBucketCols = new ArrayList<>();
       int numReduceSinks = 0;
-      int bucketingVersion = -1;
       OpTraits parentOpTraits = gbyOp.getParentOperators().get(0).getOpTraits();
       if (parentOpTraits != null) {
         numReduceSinks = parentOpTraits.getNumReduceSinks();
-        bucketingVersion = parentOpTraits.getBucketingVersion();
       }
       listBucketCols.add(gbyKeys);
       OpTraits opTraits = new OpTraits(listBucketCols, -1, listBucketCols,
-              numReduceSinks, bucketingVersion);
+          numReduceSinks);
       gbyOp.setOpTraits(opTraits);
       return null;
     }
@@ -313,16 +308,14 @@ public class OpTraitsRulesProcFactory {
 
       List<List<String>> listBucketCols = new ArrayList<>();
       int numReduceSinks = 0;
-      int bucketingVersion = -1;
       OpTraits parentOptraits = ptfOp.getParentOperators().get(0).getOpTraits();
       if (parentOptraits != null) {
         numReduceSinks = parentOptraits.getNumReduceSinks();
-        bucketingVersion = parentOptraits.getBucketingVersion();
       }
 
       listBucketCols.add(partitionKeys);
       OpTraits opTraits = new OpTraits(listBucketCols, -1, listBucketCols,
-          numReduceSinks, bucketingVersion);
+          numReduceSinks);
       ptfOp.setOpTraits(opTraits);
       return null;
     }
@@ -392,7 +385,6 @@ public class OpTraitsRulesProcFactory {
 
       int numBuckets = -1;
       int numReduceSinks = 0;
-      int bucketingVersion = -1;
       OpTraits parentOpTraits = selOp.getParentOperators().get(0).getOpTraits();
       if (parentOpTraits != null) {
         // if bucket columns are empty, then numbuckets must be set to -1.
@@ -401,10 +393,9 @@ public class OpTraitsRulesProcFactory {
           numBuckets = parentOpTraits.getNumBuckets();
         }
         numReduceSinks = parentOpTraits.getNumReduceSinks();
-        bucketingVersion = parentOpTraits.getBucketingVersion();
       }
       OpTraits opTraits = new OpTraits(listBucketCols, numBuckets, listSortCols,
-              numReduceSinks, bucketingVersion);
+          numReduceSinks);
       selOp.setOpTraits(opTraits);
       return null;
     }
@@ -442,7 +433,7 @@ public class OpTraitsRulesProcFactory {
       // The bucketingVersion is not relevant here as it is never used.
       // For SMB, we look at the parent tables' bucketing versions and for
       // bucket map join the big table's bucketing version is considered.
-      joinOp.setOpTraits(new OpTraits(bucketColsList, -1, bucketColsList, numReduceSinks, 2));
+      joinOp.setOpTraits(new OpTraits(bucketColsList, -1, bucketColsList, numReduceSinks));
       return null;
     }
 
@@ -496,8 +487,6 @@ public class OpTraitsRulesProcFactory {
       Operator<? extends OperatorDesc> operator = (Operator<? extends OperatorDesc>) nd;
 
       int numReduceSinks = 0;
-      int bucketingVersion = -1;
-      boolean bucketingVersionSeen = false;
       for (Operator<?> parentOp : operator.getParentOperators()) {
         if (parentOp.getOpTraits() == null) {
           continue;
@@ -505,17 +494,9 @@ public class OpTraitsRulesProcFactory {
         if (parentOp.getOpTraits().getNumReduceSinks() > numReduceSinks) {
           numReduceSinks = parentOp.getOpTraits().getNumReduceSinks();
         }
-        // If there is mismatch in bucketingVersion, then it should be set to
-        // -1, that way SMB will be disabled.
-        if (bucketingVersion == -1 && !bucketingVersionSeen) {
-          bucketingVersion = parentOp.getOpTraits().getBucketingVersion();
-          bucketingVersionSeen = true;
-        } else if (bucketingVersion != parentOp.getOpTraits().getBucketingVersion()) {
-          bucketingVersion = -1;
-        }
       }
       OpTraits opTraits = new OpTraits(null, -1,
-              null, numReduceSinks, bucketingVersion);
+          null, numReduceSinks);
       operator.setOpTraits(opTraits);
       return null;
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
index 29c172f..a690cd7 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/physical/Vectorizer.java
@@ -4022,9 +4022,6 @@ public class Vectorizer implements PhysicalPlanResolver {
 
     LOG.info("Vectorizer vectorizeOperator reduce sink class " + opClass.getSimpleName());
 
-    // Get the bucketing version
-    int bucketingVersion = ((ReduceSinkOperator)op).getBucketingVersion();
-
     Operator<? extends OperatorDesc> vectorOp = null;
     try {
       vectorOp = OperatorFactory.getVectorOperator(
@@ -4036,9 +4033,7 @@ public class Vectorizer implements PhysicalPlanResolver {
       throw new HiveException(e);
     }
 
-    // Set the bucketing version
     Preconditions.checkArgument(vectorOp instanceof VectorReduceSinkCommonOperator);
-    vectorOp.setBucketingVersion(bucketingVersion);
 
     return vectorOp;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
index 0638caf..bbfb853 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/optimizer/spark/SparkMapJoinOptimizer.java
@@ -121,7 +121,7 @@ public class SparkMapJoinOptimizer implements SemanticNodeProcessor {
 
     // we can set the traits for this join operator
     OpTraits opTraits = new OpTraits(bucketColNames, numBuckets, null,
-            joinOp.getOpTraits().getNumReduceSinks(), joinOp.getOpTraits().getBucketingVersion());
+        joinOp.getOpTraits().getNumReduceSinks());
     mapJoinOp.setOpTraits(opTraits);
     mapJoinOp.setStatistics(joinOp.getStatistics());
     setNumberOfBucketsOnChildren(mapJoinOp);
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 4f1e23d..19eb1df 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
@@ -54,6 +54,7 @@ import org.apache.hadoop.hive.metastore.api.SQLForeignKey;
 import org.apache.hadoop.hive.metastore.api.SQLNotNullConstraint;
 import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey;
 import org.apache.hadoop.hive.metastore.api.SQLUniqueConstraint;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.utils.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.CompilationOpContext;
 import org.apache.hadoop.hive.ql.Context;
@@ -1758,6 +1759,7 @@ public abstract class BaseSemanticAnalyzer {
     prop.setProperty("columns", colTypes[0]);
     prop.setProperty("columns.types", colTypes[1]);
     prop.setProperty(serdeConstants.SERIALIZATION_LIB, LazySimpleSerDe.class.getName());
+    prop.setProperty(hive_metastoreConstants.TABLE_BUCKETING_VERSION, "-1");
     FetchWork fetch =
         new FetchWork(ctx.getResFile(), new TableDesc(TextInputFormat.class,
             IgnoreKeyTextOutputFormat.class, prop), -1);
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 8cd763a..00fb059 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
@@ -11318,9 +11318,6 @@ public class SemanticAnalyzer extends BaseSemanticAnalyzer {
       if (properties != null) {
         tsDesc.setOpProps(properties);
       }
-
-      // Set the bucketing Version
-      top.setBucketingVersion(tsDesc.getTableMetadata().getBucketingVersion());
     } else {
       rwsch = opParseCtx.get(top).getRowResolver();
       top.setChildOperators(null);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
index 91ef159..ddcd022 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
@@ -25,6 +25,7 @@ import java.util.Comparator;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.IdentityHashMap;
 import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
@@ -81,6 +82,7 @@ import org.apache.hadoop.hive.ql.lib.SemanticRule;
 import org.apache.hadoop.hive.ql.lib.RuleRegExp;
 import org.apache.hadoop.hive.ql.log.PerfLogger;
 import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.optimizer.BucketVersionPopulator;
 import org.apache.hadoop.hive.ql.optimizer.ConstantPropagate;
 import org.apache.hadoop.hive.ql.optimizer.ConstantPropagateProcCtx.ConstantPropagateOption;
 import org.apache.hadoop.hive.ql.optimizer.ConvertJoinMapJoin;
@@ -216,6 +218,9 @@ public class TezCompiler extends TaskCompiler {
     runStatsDependentOptimizations(procCtx, inputs, outputs);
     perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, "Run the optimizations that use stats for optimization");
 
+    // repopulate bucket versions; join conversion may have created some new reducesinks
+    new BucketVersionPopulator().transform(pCtx);
+
     perfLogger.PerfLogBegin(this.getClass().getName(), PerfLogger.TEZ_COMPILER);
     if(procCtx.conf.getBoolVar(ConfVars.HIVEOPTJOINREDUCEDEDUPLICATION)) {
       new ReduceSinkJoinDeDuplication().transform(procCtx.parseContext);
@@ -243,15 +248,9 @@ public class TezCompiler extends TaskCompiler {
     markOperatorsWithUnstableRuntimeStats(procCtx);
     perfLogger.PerfLogEnd(this.getClass().getName(), PerfLogger.TEZ_COMPILER, "markOperatorsWithUnstableRuntimeStats");
 
-    // ATTENTION : DO NOT, I REPEAT, DO NOT WRITE ANYTHING AFTER updateBucketingVersionForUpgrade()
-    // ANYTHING WHICH NEEDS TO BE ADDED MUST BE ADDED ABOVE
-    // This call updates the bucketing version of final ReduceSinkOp based on
-    // the bucketing version of FileSinkOp. This operation must happen at the
-    // end to ensure there is no further rewrite of plan which may end up
-    // removing/updating the ReduceSinkOp as was the case with SortedDynPartitionOptimizer
-    // Update bucketing version of ReduceSinkOp if needed
-    updateBucketingVersionForUpgrade(procCtx);
-
+    if (procCtx.conf.getBoolVar(ConfVars.HIVE_IN_TEST)) {
+      bucketingVersionSanityCheck(procCtx);
+    }
   }
 
   private void runCycleAnalysisForPartitionPruning(OptimizeTezProcContext procCtx,
@@ -2089,7 +2088,7 @@ public class TezCompiler extends TaskCompiler {
     }
   }
 
-  private void updateBucketingVersionForUpgrade(OptimizeTezProcContext procCtx) {
+  private void bucketingVersionSanityCheck(OptimizeTezProcContext procCtx) throws SemanticException {
     // Fetch all the FileSinkOperators.
     Set<FileSinkOperator> fsOpsAll = new HashSet<>();
     for (TableScanOperator ts : procCtx.parseContext.getTopOps().values()) {
@@ -2098,7 +2097,7 @@ public class TezCompiler extends TaskCompiler {
       fsOpsAll.addAll(fsOps);
     }
 
-
+    Map<Operator<?>, Integer> processedOperators = new IdentityHashMap<>();
     for (FileSinkOperator fsOp : fsOpsAll) {
       // Look for direct parent ReduceSinkOp
       // If there are more than 1 parent, bail out.
@@ -2111,8 +2110,21 @@ public class TezCompiler extends TaskCompiler {
           continue;
         }
 
-        // Found the target RSOp
-        parent.setBucketingVersion(fsOp.getConf().getTableInfo().getBucketingVersion());
+        // Found the target RSOp 0
+        int bucketingVersion = fsOp.getConf().getTableInfo().getBucketingVersion();
+        if (fsOp.getConf().getTableInfo().getBucketingVersion() == -1) {
+          break;
+        }
+        if (fsOp.getConf().getTableInfo().getBucketingVersion() != fsOp.getConf().getBucketingVersion()) {
+          throw new RuntimeException("FsOp bucketingVersions is inconsistent with its tableinfo");
+        }
+        if (processedOperators.containsKey(parent) && processedOperators.get(parent) != bucketingVersion) {
+          throw new SemanticException(String.format(
+              "Operator (%s) is already processed and is using bucketingVersion(%d); so it can't be changed to %d ",
+              parent, processedOperators.get(parent), bucketingVersion));
+        }
+        processedOperators.put(parent, bucketingVersion);
+
         break;
       }
     }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
index 09fa145..7e0c9d0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/spark/SparkCompiler.java
@@ -145,17 +145,6 @@ public class SparkCompiler extends TaskCompiler {
       new ConstantPropagate(ConstantPropagateProcCtx.ConstantPropagateOption.SHORTCUT).transform(pCtx);
     }
 
-    // ATTENTION : DO NOT, I REPEAT, DO NOT WRITE ANYTHING AFTER updateBucketingVersionForUpgrade()
-    // ANYTHING WHICH NEEDS TO BE ADDED MUST BE ADDED ABOVE
-    // This call updates the bucketing version of final ReduceSinkOp based on
-    // the bucketing version of FileSinkOp. This operation must happen at the
-    // end to ensure there is no further rewrite of plan which may end up
-    // removing/updating the ReduceSinkOp as was the case with SortedDynPartitionOptimizer
-    // Update bucketing version of ReduceSinkOp if needed
-    // Note: This has been copied here from TezCompiler, change seems needed for bucketing to work
-    // properly moving forward.
-    updateBucketingVersionForUpgrade(procCtx);
-
     PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_OPERATOR_TREE);
   }
 
@@ -636,36 +625,4 @@ public class SparkCompiler extends TaskCompiler {
     PERF_LOGGER.PerfLogEnd(CLASS_NAME, PerfLogger.SPARK_OPTIMIZE_TASK_TREE);
     return;
   }
-
-  private void updateBucketingVersionForUpgrade(OptimizeSparkProcContext procCtx) {
-    // Fetch all the FileSinkOperators.
-    Set<FileSinkOperator> fsOpsAll = new HashSet<>();
-    for (TableScanOperator ts : procCtx.getParseContext().getTopOps().values()) {
-      Set<FileSinkOperator> fsOps = OperatorUtils.findOperators(
-          ts, FileSinkOperator.class);
-      fsOpsAll.addAll(fsOps);
-    }
-
-
-    for (FileSinkOperator fsOp : fsOpsAll) {
-      if (!fsOp.getConf().getTableInfo().isSetBucketingVersion()) {
-        continue;
-      }
-      // Look for direct parent ReduceSinkOp
-      // If there are more than 1 parent, bail out.
-      Operator<?> parent = fsOp;
-      List<Operator<?>> parentOps = parent.getParentOperators();
-      while (parentOps != null && parentOps.size() == 1) {
-        parent = parentOps.get(0);
-        if (!(parent instanceof ReduceSinkOperator)) {
-          parentOps = parent.getParentOperators();
-          continue;
-        }
-
-        // Found the target RSOp
-        parent.setBucketingVersion(fsOp.getConf().getTableInfo().getBucketingVersion());
-        break;
-      }
-    }
-  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/AbstractOperatorDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/AbstractOperatorDesc.java
index 65a107e..5553318 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/AbstractOperatorDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/AbstractOperatorDesc.java
@@ -38,6 +38,7 @@ public abstract class AbstractOperatorDesc implements OperatorDesc {
   protected long memNeeded = 0;
   protected long memAvailable = 0;
   protected String runtimeStatsTmpDir;
+  protected int bucketingVersion = -2;
 
   /**
    * A map of output column name to input expression map. This is used by
@@ -171,4 +172,13 @@ public abstract class AbstractOperatorDesc implements OperatorDesc {
     throw new RuntimeException();
   }
 
+  @Override
+  public int getBucketingVersion() {
+    return bucketingVersion;
+  }
+
+  @Override
+  public void setBucketingVersion(int bucketingVersion) {
+    this.bucketingVersion = bucketingVersion;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
index f55c6ae..619f68e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/FileSinkDesc.java
@@ -129,7 +129,7 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
       final List<ExprNodeDesc> partitionCols, final DynamicPartitionCtx dpCtx, Path destPath, Long mmWriteId,
       boolean isMmCtas, boolean isInsertOverwrite, boolean isQuery, boolean isCTASorCM, boolean isDirectInsert) {
     this.dirName = dirName;
-    this.tableInfo = tableInfo;
+    setTableInfo(tableInfo);
     this.compressed = compressed;
     this.destTableId = destTableId;
     this.multiFileSpray = multiFileSpray;
@@ -152,7 +152,7 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
       final boolean compressed) {
 
     this.dirName = dirName;
-    this.tableInfo = tableInfo;
+    setTableInfo(tableInfo);
     this.compressed = compressed;
     destTableId = 0;
     this.multiFileSpray = false;
@@ -268,6 +268,7 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
 
   public void setTableInfo(final TableDesc tableInfo) {
     this.tableInfo = tableInfo;
+    bucketingVersion = tableInfo.getBucketingVersion();
   }
 
   @Explain(displayName = "compressed")
@@ -616,6 +617,10 @@ public class FileSinkDesc extends AbstractOperatorDesc implements IStatsGatherDe
     return isMmCtas;
   }
 
+  @Explain(displayName = "bucketingVersion", explainLevels = { Level.EXTENDED })
+  public int getBucketingVersionForExplain() {
+    return getBucketingVersion();
+  }
   /**
    * Whether this is CREATE TABLE SELECT or CREATE MATERIALIZED VIEW statemet
    * Set by semantic analyzer this is required because CTAS/CM requires some special logic
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/OpTraits.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/OpTraits.java
index d3b62ce..246c089 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/OpTraits.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/OpTraits.java
@@ -26,16 +26,13 @@ public class OpTraits {
   private List<List<String>> sortColNames;
   private int numBuckets;
   private int numReduceSinks;
-  private int bucketingVersion;
 
   public OpTraits(List<List<String>> bucketColNames, int numBuckets,
-      List<List<String>> sortColNames, int numReduceSinks,
-                  int bucketingVersion) {
+      List<List<String>> sortColNames, int numReduceSinks) {
     this.bucketColNames = bucketColNames;
     this.numBuckets = numBuckets;
     this.sortColNames = sortColNames;
     this.numReduceSinks = numReduceSinks;
-    this.bucketingVersion = bucketingVersion;
   }
 
   public List<List<String>> getBucketColNames() {
@@ -71,17 +68,9 @@ public class OpTraits {
     return this.numReduceSinks;
   }
 
-  public void setBucketingVersion(int bucketingVersion) {
-    this.bucketingVersion = bucketingVersion;
-  }
-
-  public int getBucketingVersion() {
-    return bucketingVersion;
-  }
-
   @Override
   public String toString() {
     return "{ bucket column names: " + bucketColNames + "; sort column names: "
-        + sortColNames + "; bucket count: " + numBuckets + "; bucketing version: " + bucketingVersion + " }";
+        + sortColNames + "; bucket count: " + numBuckets + "}";
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/OperatorDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/OperatorDesc.java
index e8a5827..276c4a3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/OperatorDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/OperatorDesc.java
@@ -40,4 +40,8 @@ public interface OperatorDesc extends Serializable, Cloneable {
   public void setColumnExprMap(Map<String, ExprNodeDesc> colExprMap);
 
   void fillSignature(Map<String, Object> ret);
+
+  public void setBucketingVersion(int bucketingVersion);
+
+  public int getBucketingVersion();
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
index 980f39b..6282c8e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/PlanUtils.java
@@ -286,6 +286,7 @@ public final class PlanUtils {
       outputFormat = IgnoreKeyTextOutputFormat.class;
     }
     properties.setProperty(serdeConstants.SERIALIZATION_LIB, serdeClass.getName());
+    properties.setProperty(hive_metastoreConstants.TABLE_BUCKETING_VERSION, "-1");
     return new TableDesc(inputFormat, outputFormat, properties);
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
index 32715c9..a807fa9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ReduceSinkDesc.java
@@ -437,6 +437,7 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
     this.outputName = outputName;
   }
 
+  @Explain(displayName = "numBuckets", explainLevels = { Level.EXTENDED })
   public int getNumBuckets() {
     return numBuckets;
   }
@@ -445,6 +446,11 @@ public class ReduceSinkDesc extends AbstractOperatorDesc {
     this.numBuckets = numBuckets;
   }
 
+  @Explain(displayName = "bucketingVersion", explainLevels = { Level.EXTENDED })
+  public int getBucketingVersionForExplain() {
+    return getBucketingVersion();
+  }
+
   public List<ExprNodeDesc> getBucketCols() {
     return bucketCols;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
index 53a6036..6397e72 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/TableScanDesc.java
@@ -152,6 +152,7 @@ public class TableScanDesc extends AbstractOperatorDesc implements IStatsGatherD
     if (tblMetadata != null) {
       dbName = tblMetadata.getDbName();
       tableName = tblMetadata.getTableName();
+      numBuckets = tblMetadata.getNumBuckets();
     }
     isTranscationalTable = AcidUtils.isTransactionalTable(this.tableMetadata);
     if (isTranscationalTable) {
diff --git a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
index b6a6bab..163d439 100644
--- a/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
+++ b/ql/src/test/org/apache/hadoop/hive/ql/exec/TestUtilities.java
@@ -229,7 +229,8 @@ public class TestUtilities {
   private FileSinkDesc getFileSinkDesc(Path tempDirPath) {
     Table table = mock(Table.class);
     when(table.getNumBuckets()).thenReturn(NUM_BUCKETS);
-    FileSinkDesc conf = new FileSinkDesc(tempDirPath, null, false);
+    TableDesc tInfo = Utilities.getTableDesc("s", "string");
+    FileSinkDesc conf = new FileSinkDesc(tempDirPath, tInfo, false);
     conf.setTable(table);
     return conf;
   }
diff --git a/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q b/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q
index a8f5e17..c7167e7 100644
--- a/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q
+++ b/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q
@@ -19,7 +19,7 @@ CREATE TABLE test_table_n0 (key INT, value STRING) PARTITIONED BY (ds STRING, hr
 -- and the partition for 1 will get written in one reducer.  So hr=0 should be bucketed by key
 -- and hr=1 should not.
 
-EXPLAIN
+EXPLAIN EXTENDED
 INSERT OVERWRITE TABLE test_table_n0 PARTITION (ds = '2008-04-08', hr)
 SELECT key2, value, cast(hr as int) FROM
 (SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 2) as hr
diff --git a/ql/src/test/queries/clientpositive/murmur_hash_migration.q b/ql/src/test/queries/clientpositive/murmur_hash_migration.q
index 54207a7..c114ef6 100644
--- a/ql/src/test/queries/clientpositive/murmur_hash_migration.q
+++ b/ql/src/test/queries/clientpositive/murmur_hash_migration.q
@@ -36,14 +36,14 @@ analyze table srcbucket_mapjoin_part_n20 compute statistics for columns;
 
 
 CREATE TABLE tab_part_n11 (key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 4 BUCKETS STORED AS TEXTFILE;
-explain
+explain extended
 insert overwrite table tab_part_n11 partition (ds='2008-04-08')
   select key,value from srcbucket_mapjoin_part_n20;
 insert overwrite table tab_part_n11 partition (ds='2008-04-08')
   select key,value from srcbucket_mapjoin_part_n20;
 
 CREATE TABLE tab_n10(key int, value string) PARTITIONED BY(ds STRING) CLUSTERED BY (key) INTO 2 BUCKETS STORED AS TEXTFILE;
-explain
+explain extended
 insert overwrite table tab_n10 partition (ds='2008-04-08')
   select key,value from srcbucket_mapjoin_n18;
 insert overwrite table tab_n10 partition (ds='2008-04-08')
@@ -52,44 +52,14 @@ insert overwrite table tab_n10 partition (ds='2008-04-08')
 analyze table tab_part_n11 compute statistics for columns;
 analyze table tab_n10 compute statistics for columns;
 
-explain
+explain extended
 select t1.key, t1.value, t2.key, t2.value from srcbucket_mapjoin_n18 t1, srcbucket_mapjoin_part_n20 t2 where t1.key = t2.key order by t1.key, t1.value, t2.key, t2.value;
 select t1.key, t1.value, t2.key, t2.value from srcbucket_mapjoin_n18 t1, srcbucket_mapjoin_part_n20 t2 where t1.key = t2.key order by t1.key, t1.value, t2.key, t2.value;
 
-explain
+set hive.auto.convert.join=true;
+
+explain extended
 select t1.key, t1.value, t2.key, t2.value from tab_part_n11 t1, tab_n10 t2 where t1.key = t2.key order by t1.key, t1.value, t2.key, t2.value;
 select t1.key, t1.value, t2.key, t2.value from tab_part_n11 t1, tab_n10 t2 where t1.key = t2.key order by t1.key, t1.value, t2.key, t2.value;
 
 
-set hive.optimize.ppd=true;
-set hive.optimize.index.filter=true;
-set hive.tez.bucket.pruning=true;
-set hive.fetch.task.conversion=none;
-set hive.support.concurrency=true;
-set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
-
-
-create transactional table acid_ptn_bucket1 (a int, b int) partitioned by(ds string)
-clustered by (a) into 2 buckets stored as ORC
-TBLPROPERTIES('bucketing_version'='1', 'transactional'='true', 'transactional_properties'='default');
-
-explain extended insert into acid_ptn_bucket1 partition (ds) values(1,2,'today'),(1,3,'today'),(1,4,'yesterday'),(2,2,'yesterday'),(2,3,'today'),(2,4,'today');
-insert into acid_ptn_bucket1 partition (ds) values(1,2,'today'),(1,3,'today'),(1,4,'yesterday'),(2,2,'yesterday'),(2,3,'today'),(2,4,'today');
-
-alter table acid_ptn_bucket1 add columns(c int);
-
-insert into acid_ptn_bucket1 partition (ds) values(3,2,1000,'yesterday'),(3,3,1001,'today'),(3,4,1002,'yesterday'),(4,2,1003,'today'), (4,3,1004,'yesterday'),(4,4,1005,'today');
-select ROW__ID, * from acid_ptn_bucket1 where ROW__ID.bucketid = 536870912 and ds='today';
-select ROW__ID, * from acid_ptn_bucket1 where ROW__ID.bucketid = 536936448 and ds='today';
-
---create table s1 as select key, value from src where value > 2 group by key, value limit 10;
---create table s2 as select key, '45' from src s2 where key > 1 group by key limit 10;
-
-create table s1 (key int, value int) stored as ORC;
-create table s2 (key int, value int) stored as ORC;
-
-insert into s1 values(111, 33), (10, 45), (103, 44), (129, 34), (128, 11);
-insert into s2 values(10, 45), (100, 45), (103, 44), (110, 12), (128, 34), (117, 71);
-insert into table acid_ptn_bucket1 partition(ds='today') select key, count(value), key from (select * from s1 union all select * from s2) sub group by key;
-select ROW__ID, * from acid_ptn_bucket1 where ROW__ID.bucketid = 536870912 and ds='today';
-select ROW__ID, * from acid_ptn_bucket1 where ROW__ID.bucketid = 536936448 and ds='today';
diff --git a/ql/src/test/queries/clientpositive/murmur_hash_migration2.q b/ql/src/test/queries/clientpositive/murmur_hash_migration2.q
new file mode 100644
index 0000000..362ead7
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/murmur_hash_migration2.q
@@ -0,0 +1,44 @@
+--! qt:dataset:src
+set hive.stats.column.autogather=false;
+set hive.strict.checks.bucketing=false;
+
+set hive.mapred.mode=nonstrict;
+set hive.explain.user=false;
+set hive.auto.convert.join=true;
+set hive.auto.convert.join.noconditionaltask=true;
+set hive.auto.convert.join.noconditionaltask.size=30000;
+
+set hive.optimize.bucketingsorting=false;
+
+set hive.optimize.ppd=true;
+set hive.optimize.index.filter=true;
+set hive.tez.bucket.pruning=true;
+set hive.fetch.task.conversion=none;
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+
+
+create transactional table acid_ptn_bucket1 (a int, b int) partitioned by(ds string)
+clustered by (a) into 2 buckets stored as ORC
+TBLPROPERTIES('bucketing_version'='1', 'transactional'='true', 'transactional_properties'='default');
+
+explain extended insert into acid_ptn_bucket1 partition (ds) values(1,2,'today'),(1,3,'today'),(1,4,'yesterday'),(2,2,'yesterday'),(2,3,'today'),(2,4,'today');
+insert into acid_ptn_bucket1 partition (ds) values(1,2,'today'),(1,3,'today'),(1,4,'yesterday'),(2,2,'yesterday'),(2,3,'today'),(2,4,'today');
+
+alter table acid_ptn_bucket1 add columns(c int);
+
+insert into acid_ptn_bucket1 partition (ds) values(3,2,1000,'yesterday'),(3,3,1001,'today'),(3,4,1002,'yesterday'),(4,2,1003,'today'), (4,3,1004,'yesterday'),(4,4,1005,'today');
+select ROW__ID, * from acid_ptn_bucket1 where ROW__ID.bucketid = 536870912 and ds='today';
+select ROW__ID, * from acid_ptn_bucket1 where ROW__ID.bucketid = 536936448 and ds='today';
+
+--create table s1 as select key, value from src where value > 2 group by key, value limit 10;
+--create table s2 as select key, '45' from src s2 where key > 1 group by key limit 10;
+
+create table s1 (key int, value int) stored as ORC;
+create table s2 (key int, value int) stored as ORC;
+
+insert into s1 values(111, 33), (10, 45), (103, 44), (129, 34), (128, 11);
+insert into s2 values(10, 45), (100, 45), (103, 44), (110, 12), (128, 34), (117, 71);
+insert into table acid_ptn_bucket1 partition(ds='today') select key, count(value), key from (select * from s1 union all select * from s2) sub group by key;
+select ROW__ID, * from acid_ptn_bucket1 where ROW__ID.bucketid = 536870912 and ds='today';
+select ROW__ID, * from acid_ptn_bucket1 where ROW__ID.bucketid = 536936448 and ds='today';
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out
index bd87574..bcffe89 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_1.q.out
@@ -125,11 +125,14 @@ STAGE PLANS:
                   0 key (type: int)
                   1 key (type: int)
                 outputColumnNames: _col0, _col1, _col5, _col6
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -181,11 +184,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -242,11 +248,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -302,11 +311,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -371,11 +383,14 @@ STAGE PLANS:
                   0 key (type: int)
                   1 key (type: int)
                 outputColumnNames: _col0, _col1, _col5, _col6
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -427,11 +442,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -488,11 +506,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -548,11 +569,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out
index d902155..84f8360 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_10.q.out
@@ -98,11 +98,14 @@ STAGE PLANS:
                   0 userid (type: int), pageid (type: int), postid (type: int), type (type: string)
                   1 userid (type: int), pageid (type: int), postid (type: int), type (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col8, _col9, _col10, _col11
+                Statistics: Num rows: 14 Data size: 7356 Basic stats: PARTIAL Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: int), _col2 (type: int), _col3 (type: string), '1' (type: string), _col8 (type: int), _col9 (type: int), _col10 (type: int), _col11 (type: string), '2' (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
+                  Statistics: Num rows: 14 Data size: 7356 Basic stats: PARTIAL Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Statistics: Num rows: 14 Data size: 7356 Basic stats: PARTIAL Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_11.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_11.q.out
index 4731629..4089597 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_11.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_11.q.out
@@ -83,15 +83,20 @@ STAGE PLANS:
                   1 key (type: int)
                 outputColumnNames: _col0, _col7
                 Position of Big Table: 0
+                Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
                 BucketMapJoin: true
                 Select Operator
                   expressions: _col0 (type: int), _col7 (type: string)
                   outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: _col0 (type: int)
                     null sort order: a
+                    numBuckets: -1
                     sort order: +
                     Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
                     value expressions: _col1 (type: string)
                     auto parallelism: false
@@ -156,12 +161,15 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string)
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 1
 #### A masked pattern was here ####
             NumFilesPerFileSink: 16
             Static Partition Specification: ds=1/
+            Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
@@ -190,13 +198,16 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: int), _col1 (type: string)
             outputColumnNames: key, value
+            Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
               aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
               keys: '1' (type: string)
               minReductionHashAggr: 0.99
               mode: hash
               outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
+                bucketingVersion: 1
                 compressed: false
                 GlobalTableId: 0
 #### A masked pattern was here ####
@@ -260,10 +271,13 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: '1' (type: string)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: '1' (type: string)
+              Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
               tag: -1
               value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
               auto parallelism: false
@@ -302,19 +316,24 @@ STAGE PLANS:
           keys: '1' (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 275 Data size: 1100 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), '1' (type: string)
             outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 275 Data size: 1100 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics: Num rows: 275 Data size: 1100 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2
                     columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:string
                     escape.delim \
@@ -1923,6 +1942,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 250 Data size: 69750 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1933,6 +1953,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       properties:
+                        bucketing_version -1
                         columns _col0,_col1,_col2
                         columns.types int:string:string
                         escape.delim \
@@ -2040,6 +2061,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 514 Data size: 143406 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2050,6 +2072,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       properties:
+                        bucketing_version -1
                         columns _col0,_col1,_col2
                         columns.types int:string:string
                         escape.delim \
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_12.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_12.q.out
index 1f301e8..d7bd71a 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_12.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_12.q.out
@@ -98,16 +98,20 @@ STAGE PLANS:
                   1 key (type: int)
                 outputColumnNames: _col0, _col7
                 Position of Big Table: 0
+                Statistics: Num rows: 1650 Data size: 156750 Basic stats: COMPLETE Column stats: NONE
                 BucketMapJoin: true
                 Select Operator
                   expressions: _col0 (type: int), _col7 (type: string)
                   outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 1650 Data size: 156750 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
                     NumFilesPerFileSink: 1
                     Static Partition Specification: ds=1/
+                    Statistics: Num rows: 1650 Data size: 156750 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
@@ -311,15 +315,20 @@ STAGE PLANS:
                   1 key (type: int)
                 outputColumnNames: _col0, _col1, _col7
                 Position of Big Table: 0
+                Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
                 BucketMapJoin: true
                 Select Operator
                   expressions: _col0 (type: int), concat(_col1, _col7) (type: string)
                   outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: _col0 (type: int)
                     null sort order: a
+                    numBuckets: -1
                     sort order: +
                     Map-reduce partition columns: _col0 (type: int)
+                    Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
                     value expressions: _col1 (type: string)
                     auto parallelism: false
@@ -384,12 +393,15 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string)
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 1
 #### A masked pattern was here ####
             NumFilesPerFileSink: 16
             Static Partition Specification: ds=2/
+            Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
@@ -419,13 +431,16 @@ STAGE PLANS:
           Select Operator
             expressions: _col0 (type: int), _col1 (type: string)
             outputColumnNames: key, value
+            Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
               aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
               keys: '2' (type: string)
               minReductionHashAggr: 0.99
               mode: hash
               outputColumnNames: _col0, _col1, _col2
+              Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
               File Output Operator
+                bucketingVersion: 1
                 compressed: false
                 GlobalTableId: 0
 #### A masked pattern was here ####
@@ -490,10 +505,13 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: '2' (type: string)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: '2' (type: string)
+              Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
               tag: -1
               value expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
               auto parallelism: false
@@ -532,19 +550,24 @@ STAGE PLANS:
           keys: '2' (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1611 Data size: 305030 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col1 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), '2' (type: string)
             outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1611 Data size: 305030 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics: Num rows: 1611 Data size: 305030 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2
                     columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:string
                     escape.delim \
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_13.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_13.q.out
index 71ea244..70033d8 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_13.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_13.q.out
@@ -97,14 +97,19 @@ STAGE PLANS:
                   1 value (type: int)
                 outputColumnNames: _col0, _col1, _col5, _col6
                 Position of Big Table: 0
+                Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                 BucketMapJoin: true
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: _col0 (type: int)
                     null sort order: z
+                    numBuckets: -1
                     sort order: +
+                    Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
                     TopN: 10
                     TopN Hash Memory Usage: 0.1
@@ -172,18 +177,23 @@ STAGE PLANS:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: string), VALUE._col1 (type: int), VALUE._col2 (type: string)
           outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
           Limit
             Number of rows: 10
+            Statistics: Num rows: 10 Data size: 950 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
               NumFilesPerFileSink: 1
+              Statistics: Num rows: 10 Data size: 950 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3
                     columns.types int:string:int:string
                     escape.delim \
@@ -289,8 +299,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: _col0 (type: int)
                     null sort order: z
+                    numBuckets: -1
                     sort order: +
                     Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -368,6 +380,7 @@ STAGE PLANS:
             Number of rows: 10
             Statistics: Num rows: 10 Data size: 950 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -378,6 +391,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3
                     columns.types int:string:int:string
                     escape.delim \
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_16.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_16.q.out
index 1112bd3..acffa26 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_16.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_16.q.out
@@ -65,22 +65,27 @@ STAGE PLANS:
                 keys:
                   0 key (type: int)
                   1 key (type: int)
+                Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     null sort order: 
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_2.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_2.q.out
index 626691b..57793bd 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_2.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_2.q.out
@@ -80,11 +80,14 @@ STAGE PLANS:
                   0 key (type: int)
                   1 key (type: int)
                 outputColumnNames: _col0, _col1, _col5, _col6
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -138,11 +141,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -199,11 +205,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -261,11 +270,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -330,11 +342,14 @@ STAGE PLANS:
                   0 key (type: int)
                   1 key (type: int)
                 outputColumnNames: _col0, _col1, _col5, _col6
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -388,11 +403,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -449,11 +467,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -511,11 +532,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_3.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_3.q.out
index b9b0e21..520a0d6 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_3.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_3.q.out
@@ -80,11 +80,14 @@ STAGE PLANS:
                   0 key (type: int)
                   1 key (type: int)
                 outputColumnNames: _col0, _col1, _col5, _col6
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -138,11 +141,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -198,11 +204,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -260,11 +269,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -328,11 +340,14 @@ STAGE PLANS:
                   0 key (type: int)
                   1 key (type: int)
                 outputColumnNames: _col0, _col1, _col5, _col6
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                   outputColumnNames: _col0, _col1, _col2, _col3
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -386,11 +401,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -446,11 +464,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -508,11 +529,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_7.q.out b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_7.q.out
index 195bd8b..5b47efa 100644
--- a/ql/src/test/results/clientpositive/beeline/smb_mapjoin_7.q.out
+++ b/ql/src/test/results/clientpositive/beeline/smb_mapjoin_7.q.out
@@ -644,11 +644,14 @@ STAGE PLANS:
                 0 key (type: int)
                 1 key (type: int)
               outputColumnNames: _col0, _col1, _col5, _col6
+              Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col1 (type: string), _col5 (type: int), _col6 (type: string)
                 outputColumnNames: _col0, _col1, _col2, _col3
+                Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
+                  Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                   table:
                       input format: org.apache.hadoop.mapred.TextInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -657,22 +660,27 @@ STAGE PLANS:
                 Select Operator
                   expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int), _col3 (type: string)
                   outputColumnNames: k1, v1, k2, v2
+                  Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: compute_stats(k1, 'hll'), compute_stats(v1, 'hll'), compute_stats(k2, 'hll'), compute_stats(v2, 'hll')
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3
+                    Statistics: Num rows: 1 Data size: 1728 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       null sort order: 
                       sort order: 
+                      Statistics: Num rows: 1 Data size: 1728 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1), compute_stats(VALUE._col2), compute_stats(VALUE._col3)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3
+          Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/binary_output_format.q.out b/ql/src/test/results/clientpositive/binary_output_format.q.out
index ec6a3a2..b414360 100644
--- a/ql/src/test/results/clientpositive/binary_output_format.q.out
+++ b/ql/src/test/results/clientpositive/binary_output_format.q.out
@@ -80,6 +80,7 @@ STAGE PLANS:
                     input format: org.apache.hadoop.mapred.TextInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
                     properties:
+                      bucketing_version -1
                       columns _col0
                       columns.types string
                       field.delim 9
@@ -89,6 +90,7 @@ STAGE PLANS:
                     serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 1
 #### A masked pattern was here ####
@@ -133,7 +135,9 @@ STAGE PLANS:
                     outputColumnNames: _col0
                     Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       null sort order: 
+                      numBuckets: -1
                       sort order: 
                       Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: COMPLETE
                       tag: -1
@@ -200,6 +204,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 440 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -210,6 +215,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -277,6 +283,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -370,6 +377,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_1.q.out b/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
index 98c0aa4..440345f 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_1.q.out
@@ -110,7 +110,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -182,6 +184,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -192,6 +195,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_2.q.out b/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
index 01d0999..20b2703 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_2.q.out
@@ -110,7 +110,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -182,6 +184,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -192,6 +195,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out b/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out
index 8006d5c..98a45ee 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_spark1.q.out
@@ -247,6 +247,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -290,6 +291,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         File Output Operator
+                          bucketingVersion: 1
                           compressed: false
                           GlobalTableId: 0
 #### A masked pattern was here ####
@@ -458,7 +460,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
               tag: -1
@@ -499,6 +503,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -509,6 +514,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -709,6 +715,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -752,6 +759,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         File Output Operator
+                          bucketingVersion: 1
                           compressed: false
                           GlobalTableId: 0
 #### A masked pattern was here ####
@@ -920,7 +928,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
               tag: -1
@@ -961,6 +971,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -971,6 +982,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out b/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out
index 23d704a..902c129 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_spark2.q.out
@@ -231,6 +231,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -274,6 +275,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         File Output Operator
+                          bucketingVersion: 1
                           compressed: false
                           GlobalTableId: 0
 #### A masked pattern was here ####
@@ -442,7 +444,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
               tag: -1
@@ -483,6 +487,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -493,6 +498,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -693,6 +699,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -736,6 +743,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         File Output Operator
+                          bucketingVersion: 1
                           compressed: false
                           GlobalTableId: 0
 #### A masked pattern was here ####
@@ -904,7 +912,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
               tag: -1
@@ -945,6 +955,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -955,6 +966,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_spark3.q.out b/ql/src/test/results/clientpositive/bucket_map_join_spark3.q.out
index 5d59a53..42a6998 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_spark3.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_spark3.q.out
@@ -231,6 +231,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -274,6 +275,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         File Output Operator
+                          bucketingVersion: 1
                           compressed: false
                           GlobalTableId: 0
 #### A masked pattern was here ####
@@ -442,7 +444,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
               tag: -1
@@ -483,6 +487,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -493,6 +498,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -693,6 +699,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -736,6 +743,7 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1, _col2
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         File Output Operator
+                          bucketingVersion: 1
                           compressed: false
                           GlobalTableId: 0
 #### A masked pattern was here ####
@@ -904,7 +912,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
               tag: -1
@@ -945,6 +955,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -955,6 +966,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out b/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out
index 5bf5c1e..5e6a28b 100644
--- a/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out
+++ b/ql/src/test/results/clientpositive/bucket_map_join_spark4.q.out
@@ -178,6 +178,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 26 Data size: 7046 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 2
                         compressed: false
                         GlobalTableId: 0
 #### A masked pattern was here ####
@@ -188,6 +189,7 @@ STAGE PLANS:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                             properties:
+                              bucketing_version -1
                               columns _col0,_col1,_col2,_col3
                               columns.types int:string:string:string
                               escape.delim \
@@ -561,6 +563,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 26 Data size: 7046 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 2
                         compressed: false
                         GlobalTableId: 0
 #### A masked pattern was here ####
@@ -571,6 +574,7 @@ STAGE PLANS:
                             input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                             output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                             properties:
+                              bucketing_version -1
                               columns _col0,_col1,_col2,_col3
                               columns.types int:string:string:string
                               escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketcontext_1.q.out b/ql/src/test/results/clientpositive/bucketcontext_1.q.out
index b136f29..35b6ae8 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_1.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_1.q.out
@@ -230,7 +230,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -353,6 +355,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -363,6 +366,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -438,15 +442,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 Position of Big Table: 1
+                Statistics: Num rows: 250 Data size: 165502 Basic stats: PARTIAL Column stats: NONE
                 BucketMapJoin: true
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -562,16 +571,20 @@ STAGE PLANS:
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketcontext_2.q.out b/ql/src/test/results/clientpositive/bucketcontext_2.q.out
index 139b04b..442e93b 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_2.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_2.q.out
@@ -214,7 +214,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -337,6 +339,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -347,6 +350,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -422,15 +426,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 Position of Big Table: 1
+                Statistics: Num rows: 125 Data size: 82847 Basic stats: PARTIAL Column stats: NONE
                 BucketMapJoin: true
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -546,16 +555,20 @@ STAGE PLANS:
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketcontext_3.q.out b/ql/src/test/results/clientpositive/bucketcontext_3.q.out
index dbe68ee..8fa490d 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_3.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_3.q.out
@@ -262,7 +262,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -334,6 +336,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -344,6 +347,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -419,15 +423,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 Position of Big Table: 1
+                Statistics: Num rows: 125 Data size: 82847 Basic stats: PARTIAL Column stats: NONE
                 BucketMapJoin: true
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -492,16 +501,20 @@ STAGE PLANS:
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketcontext_4.q.out b/ql/src/test/results/clientpositive/bucketcontext_4.q.out
index 0cc4d7e..c0f1017 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_4.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_4.q.out
@@ -278,7 +278,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -350,6 +352,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -360,6 +363,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -435,15 +439,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 Position of Big Table: 1
+                Statistics: Num rows: 250 Data size: 165502 Basic stats: PARTIAL Column stats: NONE
                 BucketMapJoin: true
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -508,16 +517,20 @@ STAGE PLANS:
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketcontext_5.q.out b/ql/src/test/results/clientpositive/bucketcontext_5.q.out
index 25f89a8..ac4496c 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_5.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_5.q.out
@@ -140,7 +140,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -212,6 +214,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -222,6 +225,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -285,15 +289,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 Position of Big Table: 1
+                Statistics: Num rows: 1 Data size: 202 Basic stats: COMPLETE Column stats: NONE
                 BucketMapJoin: true
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -358,16 +367,20 @@ STAGE PLANS:
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketcontext_6.q.out b/ql/src/test/results/clientpositive/bucketcontext_6.q.out
index 0a5baeb..39ed5cc 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_6.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_6.q.out
@@ -162,7 +162,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -285,6 +287,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -295,6 +298,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -366,15 +370,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 Position of Big Table: 1
+                Statistics: Num rows: 117 Data size: 78681 Basic stats: PARTIAL Column stats: NONE
                 BucketMapJoin: true
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -490,16 +499,20 @@ STAGE PLANS:
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketcontext_7.q.out b/ql/src/test/results/clientpositive/bucketcontext_7.q.out
index 734dd0f..eb64514 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_7.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_7.q.out
@@ -297,7 +297,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -420,6 +422,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -430,6 +433,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -509,15 +513,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 Position of Big Table: 1
+                Statistics: Num rows: 250 Data size: 165502 Basic stats: PARTIAL Column stats: NONE
                 BucketMapJoin: true
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -633,16 +642,20 @@ STAGE PLANS:
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketcontext_8.q.out b/ql/src/test/results/clientpositive/bucketcontext_8.q.out
index 8260858..245b961 100644
--- a/ql/src/test/results/clientpositive/bucketcontext_8.q.out
+++ b/ql/src/test/results/clientpositive/bucketcontext_8.q.out
@@ -297,7 +297,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -420,6 +422,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -430,6 +433,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -509,15 +513,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 Position of Big Table: 1
+                Statistics: Num rows: 250 Data size: 165502 Basic stats: PARTIAL Column stats: NONE
                 BucketMapJoin: true
                 Group By Operator
                   aggregations: count()
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
+                    Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
                     value expressions: _col0 (type: bigint)
                     auto parallelism: false
@@ -633,16 +642,20 @@ STAGE PLANS:
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: _col0
+          Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
             NumFilesPerFileSink: 1
+            Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
 #### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin10.q.out b/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
index fcf056d..c4d6070 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin10.q.out
@@ -301,7 +301,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -421,6 +423,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -431,6 +434,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin11.q.out b/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
index 7032fb5..16114c9 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin11.q.out
@@ -317,7 +317,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -437,6 +439,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -447,6 +450,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -671,7 +675,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -791,6 +797,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -801,6 +808,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin12.q.out b/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
index 2ef7cb7..5c45394 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin12.q.out
@@ -226,7 +226,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -296,6 +298,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -306,6 +309,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -466,7 +470,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -536,6 +542,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -546,6 +553,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin13.q.out b/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
index b6e55b3..693377f 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin13.q.out
@@ -189,7 +189,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -311,6 +313,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -321,6 +324,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -493,7 +497,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -564,6 +570,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -574,6 +581,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -752,7 +760,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -823,6 +833,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -833,6 +844,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -1011,7 +1023,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                     tag: -1
@@ -1082,6 +1096,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1092,6 +1107,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin5.q.out b/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
index 5ada6e7..b0e2931 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
@@ -253,6 +253,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 312 Data size: 178025 Basic stats: PARTIAL Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -296,7 +297,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         tag: -1
@@ -416,6 +419,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -426,6 +430,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -492,6 +497,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -582,6 +588,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -868,6 +875,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 163 Data size: 93968 Basic stats: PARTIAL Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -911,7 +919,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         tag: -1
@@ -1031,6 +1041,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1041,6 +1052,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -1107,6 +1119,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1197,6 +1210,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin8.q.out b/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
index 5c0ac98..e1b658e 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin8.q.out
@@ -192,7 +192,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -262,6 +264,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -272,6 +275,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -449,7 +453,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -519,6 +525,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -529,6 +536,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin9.q.out b/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
index 61be706..709c780 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin9.q.out
@@ -192,7 +192,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -262,6 +264,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -272,6 +275,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
@@ -474,7 +478,9 @@ STAGE PLANS:
                   outputColumnNames: _col0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     null sort order: 
+                    numBuckets: -1
                     sort order: 
                     Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
                     tag: -1
@@ -544,6 +550,7 @@ STAGE PLANS:
           outputColumnNames: _col0
           Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -554,6 +561,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0
                   columns.types bigint
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out b/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
index cb9d5d0..c90fa59 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
@@ -192,6 +192,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 146 Data size: 70215 Basic stats: PARTIAL Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -235,7 +236,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         tag: -1
@@ -304,6 +307,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -314,6 +318,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -380,6 +385,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -470,6 +476,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out b/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
index b5aceed..1adac25 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
@@ -258,6 +258,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 163 Data size: 93968 Basic stats: PARTIAL Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -301,7 +302,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
                         tag: -1
@@ -370,6 +373,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -380,6 +384,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -446,6 +451,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -536,6 +542,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out b/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out
index 7b2176a..2958ae1 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin_negative3.q.out
@@ -213,6 +213,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -223,6 +224,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
@@ -370,6 +372,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -380,6 +383,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
@@ -519,6 +523,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -529,6 +534,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
@@ -670,6 +676,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -680,6 +687,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
@@ -821,6 +829,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -831,6 +840,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
@@ -972,6 +982,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -982,6 +993,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
@@ -1123,6 +1135,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1133,6 +1146,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
@@ -1274,6 +1288,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1284,6 +1299,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
@@ -1425,6 +1441,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 1 Data size: 404 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1435,6 +1452,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1,_col2,_col3
                           columns.types string:string:string:string
                           escape.delim \
diff --git a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
index e85c5c9..da8208d 100644
--- a/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/cbo_rp_auto_join1.q.out
@@ -686,23 +686,29 @@ STAGE PLANS:
                   keys:
                     0 key (type: int)
                     1 key (type: int)
+                  Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         null sort order: 
                         sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: $f0
+          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -788,23 +794,29 @@ STAGE PLANS:
                   keys:
                     0 key (type: int)
                     1 key (type: int)
+                  Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         null sort order: 
                         sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: $f0
+          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -912,23 +924,29 @@ STAGE PLANS:
                   keys:
                     0 key (type: int)
                     1 key (type: int)
+                  Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         null sort order: 
                         sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: $f0
+          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1026,23 +1044,29 @@ STAGE PLANS:
                   keys:
                     0 key (type: int)
                     1 key (type: int)
+                  Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    Statistics: Num rows: 9 Data size: 39 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         null sort order: 
                         sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: $f0
+          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1322,23 +1346,29 @@ STAGE PLANS:
                   keys:
                     0 key (type: int)
                     1 key (type: int)
+                  Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         null sort order: 
                         sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: $f0
+          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1418,23 +1448,29 @@ STAGE PLANS:
                     0 key (type: int)
                     1 key (type: int)
                     2 key (type: int)
+                  Statistics: Num rows: 15 Data size: 61 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    Statistics: Num rows: 15 Data size: 61 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         null sort order: 
                         sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: $f0
+          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1528,23 +1564,29 @@ STAGE PLANS:
                   keys:
                     0 key (type: int)
                     1 key (type: int)
+                  Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
+                    Statistics: Num rows: 7 Data size: 30 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: count()
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         null sort order: 
                         sort order: 
+                        Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col0 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
           aggregations: count(VALUE._col0)
           mode: mergepartial
           outputColumnNames: $f0
+          Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1623,11 +1665,14 @@ STAGE PLANS:
                     0 key (type: int)
                     1 key (type: int)
                   outputColumnNames: key, value, value0
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: int), value (type: string), value0 (type: string)
                     outputColumnNames: key, val1, val2
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -1707,11 +1752,14 @@ STAGE PLANS:
                     0 key (type: int)
                     1 key (type: int)
                   outputColumnNames: key, value, value0
+                  Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE
                   Select Operator
                     expressions: key (type: int), value (type: string), value0 (type: string)
                     outputColumnNames: key, val1, val2
+                    Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
                       compressed: false
+                      Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: NONE
                       table:
                           input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                           output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out b/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out
index 7e3c2ca..42e9e52 100644
--- a/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidkafkamini_basic.q.out
@@ -251,8 +251,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 2 Data size: 354 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         key expressions: _col1 (type: string)
                         null sort order: z
+                        numBuckets: -1
                         sort order: +
                         Map-reduce partition columns: _col1 (type: string)
                         Statistics: Num rows: 2 Data size: 354 Basic stats: COMPLETE Column stats: COMPLETE
@@ -328,8 +330,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
                   GatherStats: false
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: language (type: string)
                     null sort order: z
+                    numBuckets: -1
                     sort order: +
                     Map-reduce partition columns: language (type: string)
                     Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
@@ -447,6 +451,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 2 Data size: 389 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 0
                     directory: hdfs://### HDFS PATH ###
@@ -457,6 +462,7 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         properties:
+                          bucketing_version -1
                           columns _col0,_col1
                           columns.types string:string
                           escape.delim \
diff --git a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
index a993c7b..a8af291 100644
--- a/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
+++ b/ql/src/test/results/clientpositive/druid/druidmini_expressions.q.out
@@ -221,7 +221,9 @@ STAGE PLANS:
                       outputColumnNames: _col0
                       Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE
                         tag: -1
@@ -312,6 +314,7 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 1 Data size: 8 Basic stats: PARTIAL Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 0
                   directory: hdfs://### HDFS PATH ###
@@ -322,6 +325,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       properties:
+                        bucketing_version -1
                         columns _col0
                         columns.types bigint
                         escape.delim \
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
index 005e711..d6115d8 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_join_unencrypted_tbl.q.out
@@ -571,8 +571,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col2 (type: double)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col2 (type: double)
                   Statistics: Num rows: 500 Data size: 93000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -593,8 +595,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col2 (type: double)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col2 (type: double)
                   Statistics: Num rows: 500 Data size: 51500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -723,6 +727,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3
             Statistics: Num rows: 791 Data size: 215943 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
               directory: hdfs://### HDFS PATH ###
@@ -733,6 +738,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3
                     columns.types string:string:int:string
                     escape.delim \
diff --git a/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out b/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
index fb7b0bc..f6c7d50 100644
--- a/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
+++ b/ql/src/test/results/clientpositive/encrypted/encryption_join_with_different_encryption_keys.q.out
@@ -89,8 +89,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 475 Data size: 85013 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 475 Data size: 85013 Basic stats: COMPLETE Column stats: NONE
@@ -111,8 +113,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 475 Data size: 85013 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 475 Data size: 85013 Basic stats: COMPLETE Column stats: NONE
@@ -237,6 +241,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 522 Data size: 93514 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
             directory: hdfs://### HDFS PATH ###
@@ -247,6 +252,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types int:string:int:string
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/explain_rearrange.q.out b/ql/src/test/results/clientpositive/explain_rearrange.q.out
index afcd5b4..ad49f48 100644
--- a/ql/src/test/results/clientpositive/explain_rearrange.q.out
+++ b/ql/src/test/results/clientpositive/explain_rearrange.q.out
@@ -90,17 +90,20 @@ STAGE PLANS:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: int)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -108,6 +111,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -143,9 +147,11 @@ STAGE PLANS:
                 0 _col0 (type: int)
                 1 _col0 (type: int)
               outputColumnNames: _col0, _col1, _col3
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -164,13 +170,16 @@ STAGE PLANS:
               key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint)
               null sort order: zzz
               sort order: +++
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: bigint)
           outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -201,9 +210,11 @@ STAGE PLANS:
                 0 _col0 (type: int)
                 1 _col0 (type: int)
               outputColumnNames: _col0, _col1, _col3
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -223,6 +234,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: bigint)
           TableScan
             Reduce Output Operator
@@ -230,6 +242,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Join Operator
@@ -239,9 +252,11 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1, _col3
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
             outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -270,17 +285,20 @@ STAGE PLANS:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: int)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -288,6 +306,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -377,17 +396,20 @@ STAGE PLANS:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: int)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -395,6 +417,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -430,9 +453,11 @@ STAGE PLANS:
                 0 _col0 (type: int)
                 1 _col0 (type: int)
               outputColumnNames: _col0, _col1, _col3
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -451,13 +476,16 @@ STAGE PLANS:
               key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint)
               null sort order: zzz
               sort order: +++
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: bigint)
           outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -488,9 +516,11 @@ STAGE PLANS:
                 0 _col0 (type: int)
                 1 _col0 (type: int)
               outputColumnNames: _col0, _col1, _col3
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -510,6 +540,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: bigint)
           TableScan
             Reduce Output Operator
@@ -517,6 +548,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Join Operator
@@ -526,9 +558,11 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1, _col3
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
             outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -557,17 +591,20 @@ STAGE PLANS:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: int)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -575,6 +612,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -664,17 +702,20 @@ STAGE PLANS:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: int)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -682,6 +723,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -710,17 +752,20 @@ STAGE PLANS:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: int)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -728,6 +773,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -763,9 +809,11 @@ STAGE PLANS:
                 0 _col0 (type: int)
                 1 _col0 (type: int)
               outputColumnNames: _col0, _col1, _col3
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -801,9 +849,11 @@ STAGE PLANS:
                 0 _col0 (type: int)
                 1 _col0 (type: int)
               outputColumnNames: _col0, _col1, _col3
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -823,6 +873,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: bigint)
           TableScan
             Reduce Output Operator
@@ -830,6 +881,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Join Operator
@@ -839,9 +891,11 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1, _col3
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
             outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -857,13 +911,16 @@ STAGE PLANS:
               key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint)
               null sort order: zzz
               sort order: +++
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: bigint)
           outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -951,17 +1008,20 @@ STAGE PLANS:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: int)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -969,6 +1029,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -997,17 +1058,20 @@ STAGE PLANS:
                     0 _col0 (type: int)
                     1 _col0 (type: int)
                   outputColumnNames: _col0
+                  Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
                     aggregations: count()
                     keys: _col0 (type: int)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
                       key expressions: _col0 (type: int)
                       null sort order: z
                       sort order: +
                       Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -1015,6 +1079,7 @@ STAGE PLANS:
           keys: KEY._col0 (type: int)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
             table:
@@ -1062,6 +1127,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: bigint)
           TableScan
             Reduce Output Operator
@@ -1069,6 +1135,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Join Operator
@@ -1078,9 +1145,11 @@ STAGE PLANS:
             0 _col0 (type: int)
             1 _col0 (type: int)
           outputColumnNames: _col0, _col1, _col3
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
             outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
               table:
@@ -1099,9 +1168,11 @@ STAGE PLANS:
                 0 _col0 (type: int)
                 1 _col0 (type: int)
               outputColumnNames: _col0, _col1, _col3
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -1123,9 +1194,11 @@ STAGE PLANS:
                 0 _col0 (type: int)
                 1 _col0 (type: int)
               outputColumnNames: _col0, _col1, _col3
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
               Select Operator
                 expressions: _col0 (type: int), _col3 (type: bigint), _col1 (type: bigint)
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -1144,13 +1217,16 @@ STAGE PLANS:
               key expressions: _col0 (type: int), _col1 (type: bigint), _col2 (type: bigint)
               null sort order: zzz
               sort order: +++
+              Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
       Execution mode: vectorized
       Reduce Operator Tree:
         Select Operator
           expressions: KEY.reducesinkkey0 (type: int), KEY.reducesinkkey1 (type: bigint), KEY.reducesinkkey2 (type: bigint)
           outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
             compressed: false
+            Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/groupby_map_ppr.q.out b/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
index 952f310..621a80a 100644
--- a/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
+++ b/ql/src/test/results/clientpositive/groupby_map_ppr.q.out
@@ -59,8 +59,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 500 Data size: 93500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 93500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -184,6 +186,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2
             Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -227,6 +230,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 1304 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -290,7 +294,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1304 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -332,6 +338,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -342,6 +349,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
index bd43f54..4db2458 100644
--- a/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby_map_ppr_multi_distinct.q.out
@@ -59,8 +59,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
                 Statistics: Num rows: 1000 Data size: 294000 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                   null sort order: zzz
+                  numBuckets: -1
                   sort order: +++
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1000 Data size: 294000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -184,6 +186,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
             Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -227,6 +230,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -290,7 +294,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 2152 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -332,6 +338,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 1 Data size: 2200 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -342,6 +349,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3,_col4
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector: [...]
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/groupby_ppr.q.out b/ql/src/test/results/clientpositive/groupby_ppr.q.out
index d7549d9..bb5e7e6 100644
--- a/ql/src/test/results/clientpositive/groupby_ppr.q.out
+++ b/ql/src/test/results/clientpositive/groupby_ppr.q.out
@@ -52,8 +52,10 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 1000 Data size: 178000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: _col0 (type: string), _col1 (type: string)
                 null sort order: zz
+                numBuckets: -1
                 sort order: ++
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 1000 Data size: 178000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -177,6 +179,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2
             Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -214,6 +217,7 @@ STAGE PLANS:
               outputColumnNames: key, c1, c2
               Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
+                bucketingVersion: 1
                 compressed: false
                 GlobalTableId: 0
 #### A masked pattern was here ####
@@ -277,7 +281,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 316 Data size: 86268 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -319,6 +325,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -329,6 +336,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out b/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
index 95f95b0..43f1ac8 100644
--- a/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/groupby_ppr_multi_distinct.q.out
@@ -52,8 +52,10 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 1000 Data size: 178000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                 null sort order: zzz
+                numBuckets: -1
                 sort order: +++
                 Map-reduce partition columns: _col0 (type: string)
                 Statistics: Num rows: 1000 Data size: 178000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -177,6 +179,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
             Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -214,6 +217,7 @@ STAGE PLANS:
               outputColumnNames: key, c1, c2, c3, c4
               Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
+                bucketingVersion: 1
                 compressed: false
                 GlobalTableId: 0
 #### A masked pattern was here ####
@@ -277,7 +281,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 316 Data size: 88796 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -319,6 +325,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 1 Data size: 2200 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -329,6 +336,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3,_col4
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector: [...]
                   escape.delim \
@@ -426,8 +434,10 @@ STAGE PLANS:
               outputColumnNames: $f0, $f1, $f2
               Statistics: Num rows: 1000 Data size: 459000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: $f0 (type: string), $f1 (type: string), $f2 (type: string)
                 null sort order: zzz
+                numBuckets: -1
                 sort order: +++
                 Map-reduce partition columns: $f0 (type: string)
                 Statistics: Num rows: 1000 Data size: 459000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -551,6 +561,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
             Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -588,6 +599,7 @@ STAGE PLANS:
               outputColumnNames: key, c1, c2, c3, c4
               Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
+                bucketingVersion: 1
                 compressed: false
                 GlobalTableId: 0
 #### A masked pattern was here ####
@@ -651,7 +663,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 196 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -697,6 +711,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4
             Statistics: Num rows: 1 Data size: 2200 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -707,6 +722,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3,_col4
                     columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvecto [...]
                     escape.delim \
diff --git a/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out b/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
index 6498e24..8061ffa 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_1_23.q.out
@@ -82,6 +82,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -125,7 +126,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -196,6 +199,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -206,6 +210,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -272,6 +277,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -362,6 +368,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -526,8 +533,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -605,6 +614,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2
             Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -648,6 +658,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 1656 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -711,7 +722,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1656 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -753,6 +766,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1688 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -763,6 +777,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -850,6 +865,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -893,7 +909,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -964,6 +982,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -974,6 +993,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -1040,6 +1060,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1130,6 +1151,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1293,6 +1315,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -1336,7 +1359,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1407,6 +1432,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1417,6 +1443,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -1483,6 +1510,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1573,6 +1601,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1744,6 +1773,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -1787,7 +1817,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1456 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1456 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1858,6 +1890,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1504 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1868,6 +1901,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -1934,6 +1968,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2024,6 +2059,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2189,8 +2225,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -2268,6 +2306,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3
             Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -2311,6 +2350,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 2080 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2374,7 +2414,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 2080 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -2416,6 +2458,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 2128 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2426,6 +2469,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -2507,8 +2551,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: double)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: double)
                   Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
@@ -2586,6 +2632,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2
             Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -2629,6 +2676,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 1456 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2692,7 +2740,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1456 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -2734,6 +2784,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1504 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2744,6 +2795,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -2837,8 +2889,10 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       key expressions: _col0 (type: double)
                       null sort order: z
+                      numBuckets: -1
                       sort order: +
                       Map-reduce partition columns: _col0 (type: double)
                       Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -2916,6 +2970,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -2959,6 +3014,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3022,7 +3078,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -3064,6 +3122,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3074,6 +3133,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -3177,6 +3237,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -3220,7 +3281,9 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          bucketingVersion: 2
                           null sort order: 
+                          numBuckets: -1
                           sort order: 
                           Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -3247,6 +3310,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -3290,7 +3354,9 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          bucketingVersion: 2
                           null sort order: 
+                          numBuckets: -1
                           sort order: 
                           Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -3361,6 +3427,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3371,6 +3438,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -3437,6 +3505,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3527,6 +3596,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3713,8 +3783,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: double)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: double)
                   Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
@@ -3792,6 +3864,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 1
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3834,6 +3907,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -3877,7 +3951,9 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          bucketingVersion: 2
                           null sort order: 
+                          numBuckets: -1
                           sort order: 
                           Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -3892,6 +3968,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 1
 #### A masked pattern was here ####
@@ -3935,7 +4012,9 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       null sort order: 
+                      numBuckets: -1
                       sort order: 
                       Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -4029,6 +4108,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4039,6 +4119,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -4105,6 +4186,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4195,6 +4277,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4378,8 +4461,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -4402,8 +4487,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -4482,6 +4569,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 3 Data size: 607 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -4525,6 +4613,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4588,7 +4677,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -4630,6 +4721,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4640,6 +4732,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -4738,8 +4831,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
                   Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -4813,6 +4908,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4850,8 +4946,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -4861,8 +4959,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: string)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
@@ -4960,6 +5060,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 3 Data size: 607 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4970,6 +5071,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3,_col4
                   columns.types string:bigint:string:string:bigint
                   escape.delim \
@@ -5050,8 +5152,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5129,6 +5233,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -5172,6 +5277,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5235,7 +5341,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -5277,6 +5385,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5287,6 +5396,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -5372,6 +5482,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 6 Data size: 588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -5415,7 +5526,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -5486,6 +5599,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5496,6 +5610,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -5562,6 +5677,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5652,6 +5768,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5826,6 +5943,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 6 Data size: 612 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -5869,7 +5987,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -5940,6 +6060,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 1 Data size: 2200 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5950,6 +6071,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3,_col4
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:stru [...]
                   escape.delim \
@@ -6016,6 +6138,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6106,6 +6229,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6279,6 +6403,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 6 Data size: 588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -6322,7 +6447,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -6393,6 +6520,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6403,6 +6531,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -6469,6 +6598,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6559,6 +6689,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6739,6 +6870,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 6 Data size: 588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -6782,7 +6914,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -6853,6 +6987,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6863,6 +6998,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -6929,6 +7065,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -7019,6 +7156,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/groupby_sort_6.q.out b/ql/src/test/results/clientpositive/groupby_sort_6.q.out
index 6930641..6bf1057 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_6.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_6.q.out
@@ -61,8 +61,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: _col0 (type: string)
                     null sort order: z
+                    numBuckets: -1
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
@@ -83,6 +85,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -126,6 +129,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -189,7 +193,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -231,6 +237,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -241,6 +248,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -329,8 +337,10 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
                   Reduce Output Operator
+                    bucketingVersion: 2
                     key expressions: _col0 (type: string)
                     null sort order: z
+                    numBuckets: -1
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
                     Statistics: Num rows: 1 Data size: 92 Basic stats: COMPLETE Column stats: COMPLETE
@@ -351,6 +361,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -394,6 +405,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -457,7 +469,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -499,6 +513,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -509,6 +524,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -586,8 +602,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 184 Basic stats: PARTIAL Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 1 Data size: 184 Basic stats: PARTIAL Column stats: NONE
@@ -660,6 +678,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 1 Data size: 184 Basic stats: PARTIAL Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -703,6 +722,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 1032 Basic stats: PARTIAL Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -766,7 +786,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1032 Basic stats: PARTIAL Column stats: NONE
               tag: -1
@@ -808,6 +830,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: PARTIAL Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -818,6 +841,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out b/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
index 38826ef..0588f5f 100644
--- a/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
+++ b/ql/src/test/results/clientpositive/groupby_sort_skew_1_23.q.out
@@ -82,6 +82,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -125,7 +126,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -196,6 +199,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -206,6 +210,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -272,6 +277,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -362,6 +368,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -527,8 +534,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -602,6 +611,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -626,8 +636,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: string), _col1 (type: string)
               null sort order: zz
+              numBuckets: -1
               sort order: ++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
               Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -675,6 +687,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2
             Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -718,6 +731,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 1656 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -781,7 +795,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1656 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -823,6 +839,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1688 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -833,6 +850,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -920,6 +938,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -963,7 +982,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1034,6 +1055,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1044,6 +1066,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -1110,6 +1133,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1200,6 +1224,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1363,6 +1388,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -1406,7 +1432,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1
                       Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1477,6 +1505,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1487,6 +1516,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -1553,6 +1583,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1643,6 +1674,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1814,6 +1846,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -1857,7 +1890,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1456 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1456 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -1928,6 +1963,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1504 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1938,6 +1974,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -2004,6 +2041,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2094,6 +2132,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2260,8 +2299,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -2335,6 +2376,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2359,8 +2401,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: string), _col1 (type: string)
               null sort order: zz
+              numBuckets: -1
               sort order: ++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
               Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -2408,6 +2452,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3
             Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -2451,6 +2496,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 2080 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2514,7 +2560,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 2080 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -2556,6 +2604,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 2128 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2566,6 +2615,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -2648,8 +2698,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: double)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
@@ -2723,6 +2775,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2747,8 +2800,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: string), _col1 (type: double)
               null sort order: zz
+              numBuckets: -1
               sort order: ++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: double)
               Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
@@ -2796,6 +2851,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2
             Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -2839,6 +2895,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 1456 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2902,7 +2959,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1456 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -2944,6 +3003,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1504 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -2954,6 +3014,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -3048,8 +3109,10 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       key expressions: _col0 (type: double)
                       null sort order: z
+                      numBuckets: -1
                       sort order: +
                       Map-reduce partition columns: rand() (type: double)
                       Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -3123,6 +3186,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3147,8 +3211,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: double)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: double)
               Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -3196,6 +3262,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -3239,6 +3306,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3302,7 +3370,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -3344,6 +3414,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3354,6 +3425,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -3457,6 +3529,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -3500,7 +3573,9 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          bucketingVersion: 2
                           null sort order: 
+                          numBuckets: -1
                           sort order: 
                           Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -3527,6 +3602,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -3570,7 +3646,9 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          bucketingVersion: 2
                           null sort order: 
+                          numBuckets: -1
                           sort order: 
                           Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -3641,6 +3719,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3651,6 +3730,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -3717,6 +3797,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3807,6 +3888,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -3994,8 +4076,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: double)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
@@ -4069,6 +4153,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4093,8 +4178,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: double)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: double)
               Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
@@ -4142,6 +4229,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 1
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4184,6 +4272,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                     File Output Operator
+                      bucketingVersion: 2
                       compressed: false
                       GlobalTableId: 1
 #### A masked pattern was here ####
@@ -4227,7 +4316,9 @@ STAGE PLANS:
                         outputColumnNames: _col0, _col1
                         Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                         Reduce Output Operator
+                          bucketingVersion: 2
                           null sort order: 
+                          numBuckets: -1
                           sort order: 
                           Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                           tag: -1
@@ -4242,6 +4333,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 6 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 1
 #### A masked pattern was here ####
@@ -4285,7 +4377,9 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1
                     Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       null sort order: 
+                      numBuckets: -1
                       sort order: 
                       Statistics: Num rows: 1 Data size: 1032 Basic stats: COMPLETE Column stats: NONE
                       tag: -1
@@ -4379,6 +4473,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 1064 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4389,6 +4484,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -4455,6 +4551,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4545,6 +4642,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4728,8 +4826,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -4752,8 +4852,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -4832,6 +4934,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 3 Data size: 607 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -4875,6 +4978,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4938,7 +5042,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: NONE
               tag: -1
@@ -4980,6 +5086,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -4990,6 +5097,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -5089,8 +5197,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string), _col1 (type: string)
                   null sort order: zz
+                  numBuckets: -1
                   sort order: ++
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -5164,6 +5274,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5188,8 +5299,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: string), _col1 (type: string)
               null sort order: zz
+              numBuckets: -1
               sort order: ++
               Map-reduce partition columns: _col0 (type: string), _col1 (type: string)
               Statistics: Num rows: 6 Data size: 2208 Basic stats: COMPLETE Column stats: NONE
@@ -5233,6 +5346,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5270,8 +5384,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 3 Data size: 552 Basic stats: COMPLETE Column stats: NONE
@@ -5281,8 +5397,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: string)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 3 Data size: 1104 Basic stats: COMPLETE Column stats: NONE
@@ -5380,6 +5498,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 3 Data size: 607 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5390,6 +5509,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3,_col4
                   columns.types string:bigint:string:string:bigint
                   escape.delim \
@@ -5471,8 +5591,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: rand() (type: double)
                   Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5546,6 +5668,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5570,8 +5693,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: string)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: string)
               Statistics: Num rows: 3 Data size: 279 Basic stats: COMPLETE Column stats: COMPLETE
@@ -5619,6 +5744,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 3 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -5662,6 +5788,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5725,7 +5852,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 848 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -5767,6 +5896,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5777,6 +5907,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -5862,6 +5993,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 6 Data size: 588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -5905,7 +6037,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -5976,6 +6110,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -5986,6 +6121,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -6052,6 +6188,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6142,6 +6279,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6316,6 +6454,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 6 Data size: 612 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -6359,7 +6498,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 2136 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -6430,6 +6571,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 1 Data size: 2200 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6440,6 +6582,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3,_col4
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:stru [...]
                   escape.delim \
@@ -6506,6 +6649,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6596,6 +6740,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6769,6 +6914,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 6 Data size: 588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -6812,7 +6958,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -6883,6 +7031,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -6893,6 +7042,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -6959,6 +7109,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -7049,6 +7200,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -7229,6 +7381,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2, _col3
                   Statistics: Num rows: 6 Data size: 588 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -7272,7 +7425,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2, _col3
                       Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1712 Basic stats: COMPLETE Column stats: COMPLETE
                         tag: -1
@@ -7343,6 +7498,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -7353,6 +7509,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -7419,6 +7576,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -7509,6 +7667,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
index 9b869a1..02c40a2 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_dyn_part.q.out
@@ -661,7 +661,7 @@ Partition Parameters:
 	numFiles            	1                   
 	numRows             	305                 
 	rawDataSize         	1163                
-	totalSize           	1346                
+	totalSize           	1347                
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
index 297084d..e2234c3 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_map_operators.q.out
@@ -508,11 +508,14 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 outputColumnNames: _col0, _col6
+                Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                 Select Operator
                   expressions: _col0 (type: string), _col6 (type: string)
                   outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
                     compressed: false
+                    Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                     table:
                         input format: org.apache.hadoop.mapred.TextInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -521,17 +524,20 @@ STAGE PLANS:
                   Select Operator
                     expressions: _col0 (type: string), _col1 (type: string)
                     outputColumnNames: key, value
+                    Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
                       aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                       keys: '1' (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2
+                      Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
                         key expressions: '1' (type: string)
                         null sort order: z
                         sort order: +
                         Map-reduce partition columns: '1' (type: string)
+                        Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
       Reduce Operator Tree:
         Group By Operator
@@ -539,11 +545,14 @@ STAGE PLANS:
           keys: '1' (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col1 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), '1' (type: string)
             outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
+              Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
@@ -697,17 +706,20 @@ STAGE PLANS:
                   0 key (type: string)
                   1 key (type: string)
                 outputColumnNames: _col6
+                Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                 Group By Operator
                   aggregations: count()
                   keys: _col6 (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1
+                  Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                   Reduce Output Operator
                     key expressions: _col0 (type: string)
                     null sort order: z
                     sort order: +
                     Map-reduce partition columns: _col0 (type: string)
+                    Statistics: Num rows: 550 Data size: 47850 Basic stats: COMPLETE Column stats: NONE
                     value expressions: _col1 (type: bigint)
       Reduce Operator Tree:
         Group By Operator
@@ -715,11 +727,14 @@ STAGE PLANS:
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1
+          Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col0 (type: string), CAST( _col1 AS STRING) (type: string)
             outputColumnNames: _col0, _col1
+            Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
+              Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
               table:
                   input format: org.apache.hadoop.mapred.TextInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
@@ -728,12 +743,14 @@ STAGE PLANS:
             Select Operator
               expressions: _col0 (type: string), _col1 (type: string)
               outputColumnNames: key, value
+              Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
               Group By Operator
                 aggregations: compute_stats(key, 'hll'), compute_stats(value, 'hll')
                 keys: '1' (type: string)
                 minReductionHashAggr: 0.99
                 mode: hash
                 outputColumnNames: _col0, _col1, _col2
+                Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
                   compressed: false
                   table:
@@ -770,6 +787,7 @@ STAGE PLANS:
               null sort order: z
               sort order: +
               Map-reduce partition columns: '1' (type: string)
+              Statistics: Num rows: 275 Data size: 23925 Basic stats: COMPLETE Column stats: NONE
               value expressions: _col1 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
       Execution mode: vectorized
       Reduce Operator Tree:
@@ -778,11 +796,14 @@ STAGE PLANS:
           keys: '1' (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2
+          Statistics: Num rows: 137 Data size: 11919 Basic stats: COMPLETE Column stats: NONE
           Select Operator
             expressions: _col1 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), _col2 (type: struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>), '1' (type: string)
             outputColumnNames: _col0, _col1, _col2
+            Statistics: Num rows: 137 Data size: 11919 Basic stats: COMPLETE Column stats: NONE
             File Output Operator
               compressed: false
+              Statistics: Num rows: 137 Data size: 11919 Basic stats: COMPLETE Column stats: NONE
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out
index 52fd083..cd19964 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_num_buckets.q.out
@@ -6,7 +6,7 @@ POSTHOOK: query: CREATE TABLE test_table_n0 (key INT, value STRING) PARTITIONED
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@test_table_n0
-PREHOOK: query: EXPLAIN
+PREHOOK: query: EXPLAIN EXTENDED
 INSERT OVERWRITE TABLE test_table_n0 PARTITION (ds = '2008-04-08', hr)
 SELECT key2, value, cast(hr as int) FROM
 (SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 2) as hr
@@ -18,7 +18,7 @@ PREHOOK: Input: default@srcpart
 PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=11
 PREHOOK: Input: default@srcpart@ds=2008-04-08/hr=12
 PREHOOK: Output: default@test_table_n0@ds=2008-04-08
-POSTHOOK: query: EXPLAIN
+POSTHOOK: query: EXPLAIN EXTENDED
 INSERT OVERWRITE TABLE test_table_n0 PARTITION (ds = '2008-04-08', hr)
 SELECT key2, value, cast(hr as int) FROM
 (SELECT if ((key % 3) < 2, 0, 1) as key2, value, (key % 2) as hr
@@ -44,17 +44,127 @@ STAGE PLANS:
             alias: srcpart
             filterExpr: (ds = '2008-04-08') (type: boolean)
             Statistics: Num rows: 1000 Data size: 178000 Basic stats: COMPLETE Column stats: COMPLETE
+            GatherStats: false
             Select Operator
               expressions: if(((key % 3) < 2), 0, 1) (type: int), value (type: string), UDFToInteger((key % 2)) (type: int)
               outputColumnNames: _col0, _col1, _col2
               Statistics: Num rows: 1000 Data size: 99000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 null sort order: 
+                numBuckets: -1
                 sort order: 
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 1000 Data size: 99000 Basic stats: COMPLETE Column stats: COMPLETE
+                tag: -1
                 value expressions: _col0 (type: int), _col1 (type: string), _col2 (type: int)
+                auto parallelism: false
       Execution mode: vectorized
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: hr=11
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            partition values:
+              ds 2008-04-08
+              hr 11
+            properties:
+              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+              bucket_count -1
+              column.name.delimiter ,
+              columns key,value
+              columns.comments 'default','default'
+              columns.types string:string
+#### A masked pattern was here ####
+              name default.srcpart
+              numFiles 1
+              numRows 500
+              partition_columns ds/hr
+              partition_columns.types string:string
+              rawDataSize 5312
+              serialization.ddl struct srcpart { string key, string value}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 5812
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                bucketing_version 2
+                column.name.delimiter ,
+                columns key,value
+                columns.comments 'default','default'
+                columns.types string:string
+#### A masked pattern was here ####
+                name default.srcpart
+                partition_columns ds/hr
+                partition_columns.types string:string
+                serialization.ddl struct srcpart { string key, string value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.srcpart
+            name: default.srcpart
+#### A masked pattern was here ####
+          Partition
+            base file name: hr=12
+            input format: org.apache.hadoop.mapred.TextInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+            partition values:
+              ds 2008-04-08
+              hr 12
+            properties:
+              COLUMN_STATS_ACCURATE {"BASIC_STATS":"true","COLUMN_STATS":{"key":"true","value":"true"}}
+              bucket_count -1
+              column.name.delimiter ,
+              columns key,value
+              columns.comments 'default','default'
+              columns.types string:string
+#### A masked pattern was here ####
+              name default.srcpart
+              numFiles 1
+              numRows 500
+              partition_columns ds/hr
+              partition_columns.types string:string
+              rawDataSize 5312
+              serialization.ddl struct srcpart { string key, string value}
+              serialization.format 1
+              serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              totalSize 5812
+#### A masked pattern was here ####
+            serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+          
+              input format: org.apache.hadoop.mapred.TextInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                bucketing_version 2
+                column.name.delimiter ,
+                columns key,value
+                columns.comments 'default','default'
+                columns.types string:string
+#### A masked pattern was here ####
+                name default.srcpart
+                partition_columns ds/hr
+                partition_columns.types string:string
+                serialization.ddl struct srcpart { string key, string value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
+              serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              name: default.srcpart
+            name: default.srcpart
+      Truncated Path -> Alias:
+        /srcpart/ds=2008-04-08/hr=11 [a:srcpart]
+        /srcpart/ds=2008-04-08/hr=12 [a:srcpart]
+      Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
           expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), CAST( VALUE._col2 AS STRING) (type: string)
@@ -72,30 +182,89 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 316 Data size: 360872 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
+                bucketingVersion: 1
                 compressed: false
+                GlobalTableId: 0
+#### A masked pattern was here ####
+                NumFilesPerFileSink: 1
                 table:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                    properties:
+                      column.name.delimiter ,
+                      columns _col0,_col1,_col2,_col3
+                      columns.types string,string,struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>,struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>
+                      escape.delim \
+                      serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
                     serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+                TotalFiles: 1
+                GatherStats: false
+                MultiFileSpray: false
           File Output Operator
+            bucketingVersion: 1
             compressed: false
+            GlobalTableId: 0
+#### A masked pattern was here ####
+            NumFilesPerFileSink: 1
             table:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                properties:
+                  column.name.delimiter ,
+                  columns _col0,_col1,_col2
+                  columns.types int,string,string
+                  escape.delim \
+                  serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
                 serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            TotalFiles: 1
+            GatherStats: false
+            MultiFileSpray: false
 
   Stage: Stage-2
     Map Reduce
       Map Operator Tree:
           TableScan
+            GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: '2008-04-08' (type: string), _col1 (type: string)
               null sort order: zz
+              numBuckets: -1
               sort order: ++
               Map-reduce partition columns: '2008-04-08' (type: string), _col1 (type: string)
               Statistics: Num rows: 316 Data size: 360872 Basic stats: COMPLETE Column stats: COMPLETE
+              tag: -1
               value expressions: _col2 (type: struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>), _col3 (type: struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>)
+              auto parallelism: false
       Execution mode: vectorized
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -mr-10002
+            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+            properties:
+              column.name.delimiter ,
+              columns _col0,_col1,_col2,_col3
+              columns.types string,string,struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>,struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>
+              escape.delim \
+              serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+          
+              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+              properties:
+                column.name.delimiter ,
+                columns _col0,_col1,_col2,_col3
+                columns.types string,string,struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,bitvector:binary>,struct<columntype:string,maxlength:bigint,sumlength:bigint,count:bigint,countnulls:bigint,bitvector:binary>
+                escape.delim \
+                serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+              serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+      Truncated Path -> Alias:
+#### A masked pattern was here ####
+      Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
           aggregations: compute_stats(VALUE._col0), compute_stats(VALUE._col1)
@@ -108,46 +277,126 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3
             Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
+              GlobalTableId: 0
+#### A masked pattern was here ####
+              NumFilesPerFileSink: 1
               Statistics: Num rows: 316 Data size: 365928 Basic stats: COMPLETE Column stats: COMPLETE
+#### A masked pattern was here ####
               table:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                  properties:
+                    bucketing_version -1
+                    columns _col0,_col1,_col2,_col3
+                    columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:string:string
+                    escape.delim \
+                    hive.serialization.extend.additional.nesting.levels true
+                    serialization.escape.crlf true
+                    serialization.format 1
+                    serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                   serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+              TotalFiles: 1
+              GatherStats: false
+              MultiFileSpray: false
 
   Stage: Stage-4
     Stats Work
       Basic Stats Work:
+#### A masked pattern was here ####
       Column Stats Desc:
           Columns: key, value
           Column Types: int, string
           Table: default.test_table_n0
+          Is Table Level Stats: false
 
   Stage: Stage-3
     Map Reduce
       Map Operator Tree:
           TableScan
+            GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col2 (type: string)
               null sort order: a
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col2 (type: string)
               Statistics: Num rows: 1000 Data size: 279000 Basic stats: COMPLETE Column stats: COMPLETE
+              tag: -1
               value expressions: _col0 (type: int), _col1 (type: string)
+              auto parallelism: false
+      Path -> Bucketed Columns:
+#### A masked pattern was here ####
       Execution mode: vectorized
+      Path -> Alias:
+#### A masked pattern was here ####
+      Path -> Partition:
+#### A masked pattern was here ####
+          Partition
+            base file name: -mr-10003
+            input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+            output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+            properties:
+              column.name.delimiter ,
+              columns _col0,_col1,_col2
+              columns.types int,string,string
+              escape.delim \
+              serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+            serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+          
+              input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+              output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+              properties:
+                column.name.delimiter ,
+                columns _col0,_col1,_col2
+                columns.types int,string,string
+                escape.delim \
+                serialization.lib org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+              serde: org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe
+      Path -> Sorted Columns:
+#### A masked pattern was here ####
+      Truncated Path -> Alias:
+#### A masked pattern was here ####
+      Needs Tagging: false
       Reduce Operator Tree:
         Select Operator
           expressions: VALUE._col0 (type: int), VALUE._col1 (type: string), KEY._col2 (type: string)
           outputColumnNames: _col0, _col1, _col2
           File Output Operator
+            bucketingVersion: 2
             compressed: false
+            GlobalTableId: 1
+#### A masked pattern was here ####
             Dp Sort State: PARTITION_SORTED
+            NumFilesPerFileSink: 1
+            Static Partition Specification: ds=2008-04-08/
             Statistics: Num rows: 1000 Data size: 279000 Basic stats: COMPLETE Column stats: COMPLETE
+#### A masked pattern was here ####
             table:
                 input format: org.apache.hadoop.mapred.TextInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                properties:
+                  bucket_count -1
+                  bucketing_version 2
+                  column.name.delimiter ,
+                  columns key,value
+                  columns.comments 
+                  columns.types int:string
+#### A masked pattern was here ####
+                  name default.test_table_n0
+                  partition_columns ds/hr
+                  partition_columns.types string:string
+                  serialization.ddl struct test_table_n0 { i32 key, string value}
+                  serialization.format 1
+                  serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
                 serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                 name: default.test_table_n0
+            TotalFiles: 1
+            GatherStats: true
+            MultiFileSpray: false
 
   Stage: Stage-0
     Move Operator
@@ -156,9 +405,25 @@ STAGE PLANS:
             ds 2008-04-08
             hr 
           replace: true
+#### A masked pattern was here ####
           table:
               input format: org.apache.hadoop.mapred.TextInputFormat
               output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+              properties:
+                bucket_count -1
+                bucketing_version 2
+                column.name.delimiter ,
+                columns key,value
+                columns.comments 
+                columns.types int:string
+#### A masked pattern was here ####
+                name default.test_table_n0
+                partition_columns ds/hr
+                partition_columns.types string:string
+                serialization.ddl struct test_table_n0 { i32 key, string value}
+                serialization.format 1
+                serialization.lib org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+#### A masked pattern was here ####
               serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
               name: default.test_table_n0
 
diff --git a/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out b/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
index 53bc817..aa0d49a 100644
--- a/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
+++ b/ql/src/test/results/clientpositive/infer_bucket_sort_reducers_power_two.q.out
@@ -187,9 +187,9 @@ Table:              	test_table_n14
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
 	numFiles            	13                  
-	numRows             	259                 
-	rawDataSize         	2783                
-	totalSize           	3042                
+	numRows             	2654                
+	rawDataSize         	28466               
+	totalSize           	31120               
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
@@ -236,9 +236,9 @@ Table:              	test_table_n14
 Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\"}
 	numFiles            	16                  
-	numRows             	257                 
-	rawDataSize         	2823                
-	totalSize           	3080                
+	numRows             	2654                
+	rawDataSize         	28466               
+	totalSize           	31120               
 #### A masked pattern was here ####
 	 	 
 # Storage Information	 	 
diff --git a/ql/src/test/results/clientpositive/input23.q.out b/ql/src/test/results/clientpositive/input23.q.out
index 396f2c1..cd79dd3 100644
--- a/ql/src/test/results/clientpositive/input23.q.out
+++ b/ql/src/test/results/clientpositive/input23.q.out
@@ -37,7 +37,9 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1
               Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 null sort order: 
+                numBuckets: -1
                 sort order: 
                 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                 tag: 0
@@ -57,7 +59,9 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   null sort order: 
+                  numBuckets: -1
                   sort order: 
                   Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                   tag: 1
@@ -135,6 +139,7 @@ STAGE PLANS:
               Number of rows: 5
               Statistics: Num rows: 5 Data size: 3530 Basic stats: COMPLETE Column stats: COMPLETE
               File Output Operator
+                bucketingVersion: 2
                 compressed: false
                 GlobalTableId: 0
 #### A masked pattern was here ####
@@ -145,6 +150,7 @@ STAGE PLANS:
                     input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                     output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                     properties:
+                      bucketing_version -1
                       columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7
                       columns.types string:string:string:string:string:string:string:string
                       escape.delim \
diff --git a/ql/src/test/results/clientpositive/input42.q.out b/ql/src/test/results/clientpositive/input42.q.out
index 9b17445..15c363b 100644
--- a/ql/src/test/results/clientpositive/input42.q.out
+++ b/ql/src/test/results/clientpositive/input42.q.out
@@ -1180,6 +1180,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 333 Data size: 151848 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1190,6 +1191,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       properties:
+                        bucketing_version -1
                         columns _col0,_col1,_col2,_col3
                         columns.types string:string:string:string
                         escape.delim \
@@ -1741,6 +1743,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 333 Data size: 151848 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1751,6 +1754,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       properties:
+                        bucketing_version -1
                         columns _col0,_col1,_col2,_col3
                         columns.types string:string:string:string
                         escape.delim \
diff --git a/ql/src/test/results/clientpositive/input_part1.q.out b/ql/src/test/results/clientpositive/input_part1.q.out
index 195f52c..3ac6602 100644
--- a/ql/src/test/results/clientpositive/input_part1.q.out
+++ b/ql/src/test/results/clientpositive/input_part1.q.out
@@ -51,6 +51,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 166 Data size: 45650 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 1
 #### A masked pattern was here ####
@@ -94,7 +95,9 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2, _col3
                     Statistics: Num rows: 1 Data size: 1744 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       null sort order: 
+                      numBuckets: -1
                       sort order: 
                       Statistics: Num rows: 1 Data size: 1744 Basic stats: COMPLETE Column stats: COMPLETE
                       tag: -1
@@ -162,6 +165,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -172,6 +176,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,n [...]
                   escape.delim \
@@ -238,6 +243,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -328,6 +334,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/input_part2.q.out b/ql/src/test/results/clientpositive/input_part2.q.out
index b187bc2..9d56af7 100644
--- a/ql/src/test/results/clientpositive/input_part2.q.out
+++ b/ql/src/test/results/clientpositive/input_part2.q.out
@@ -64,6 +64,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 333 Data size: 91575 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 1
 #### A masked pattern was here ####
@@ -107,7 +108,9 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2, _col3
                     Statistics: Num rows: 1 Data size: 1744 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       null sort order: 
+                      numBuckets: -1
                       sort order: 
                       Statistics: Num rows: 1 Data size: 1744 Basic stats: COMPLETE Column stats: COMPLETE
                       tag: -1
@@ -122,6 +125,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 333 Data size: 91575 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 2
 #### A masked pattern was here ####
@@ -165,6 +169,7 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col2, _col3
                     Statistics: Num rows: 1 Data size: 1744 Basic stats: COMPLETE Column stats: COMPLETE
                     File Output Operator
+                      bucketingVersion: 1
                       compressed: false
                       GlobalTableId: 0
 #### A masked pattern was here ####
@@ -294,6 +299,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -304,6 +310,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,n [...]
                   escape.delim \
@@ -370,6 +377,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -460,6 +468,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -595,7 +604,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1744 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -637,6 +648,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -647,6 +659,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,n [...]
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/input_part7.q.out b/ql/src/test/results/clientpositive/input_part7.q.out
index 5ac50a4..f040d60 100644
--- a/ql/src/test/results/clientpositive/input_part7.q.out
+++ b/ql/src/test/results/clientpositive/input_part7.q.out
@@ -50,8 +50,10 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col3
                     Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       key expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
                       null sort order: zzz
+                      numBuckets: -1
                       sort order: +++
                       Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                       tag: -1
@@ -76,8 +78,10 @@ STAGE PLANS:
                     outputColumnNames: _col0, _col1, _col3
                     Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                     Reduce Output Operator
+                      bucketingVersion: 2
                       key expressions: _col0 (type: string), _col1 (type: string), _col3 (type: string)
                       null sort order: zzz
+                      numBuckets: -1
                       sort order: +++
                       Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                       tag: -1
@@ -193,6 +197,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -203,6 +208,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types string:string:string:string
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/input_part9.q.out b/ql/src/test/results/clientpositive/input_part9.q.out
index a15a77b..98293b1 100644
--- a/ql/src/test/results/clientpositive/input_part9.q.out
+++ b/ql/src/test/results/clientpositive/input_part9.q.out
@@ -37,6 +37,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1000 Data size: 456000 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -47,6 +48,7 @@ STAGE PLANS:
                       input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                       output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                       properties:
+                        bucketing_version -1
                         columns _col0,_col1,_col2,_col3
                         columns.types string:string:string:string
                         escape.delim \
diff --git a/ql/src/test/results/clientpositive/join17.q.out b/ql/src/test/results/clientpositive/join17.q.out
index e8bd76d..8d28f35 100644
--- a/ql/src/test/results/clientpositive/join17.q.out
+++ b/ql/src/test/results/clientpositive/join17.q.out
@@ -49,8 +49,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -71,8 +73,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -147,6 +151,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3
             Statistics: Num rows: 791 Data size: 150290 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -190,6 +195,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2, _col3
                 Statistics: Num rows: 1 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -253,7 +259,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1728 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -295,6 +303,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3
           Statistics: Num rows: 1 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -305,6 +314,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2,_col3
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/join26.q.out b/ql/src/test/results/clientpositive/join26.q.out
index aaa6bf2..2031b58 100644
--- a/ql/src/test/results/clientpositive/join26.q.out
+++ b/ql/src/test/results/clientpositive/join26.q.out
@@ -183,6 +183,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 61 Data size: 16348 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 2
                         compressed: false
                         GlobalTableId: 1
 #### A masked pattern was here ####
@@ -226,6 +227,7 @@ STAGE PLANS:
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
+                            bucketingVersion: 1
                             compressed: false
                             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -441,7 +443,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -483,6 +487,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -493,6 +498,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/join32.q.out b/ql/src/test/results/clientpositive/join32.q.out
index d45e6b9..a8b9ae9 100644
--- a/ql/src/test/results/clientpositive/join32.q.out
+++ b/ql/src/test/results/clientpositive/join32.q.out
@@ -183,6 +183,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 63 Data size: 16884 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 2
                         compressed: false
                         GlobalTableId: 1
 #### A masked pattern was here ####
@@ -226,6 +227,7 @@ STAGE PLANS:
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
+                            bucketingVersion: 1
                             compressed: false
                             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -441,7 +443,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -483,6 +487,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -493,6 +498,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/join33.q.out b/ql/src/test/results/clientpositive/join33.q.out
index 04d9951..b53a32a 100644
--- a/ql/src/test/results/clientpositive/join33.q.out
+++ b/ql/src/test/results/clientpositive/join33.q.out
@@ -183,6 +183,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 63 Data size: 16884 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 2
                         compressed: false
                         GlobalTableId: 1
 #### A masked pattern was here ####
@@ -226,6 +227,7 @@ STAGE PLANS:
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
+                            bucketingVersion: 1
                             compressed: false
                             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -441,7 +443,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -483,6 +487,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -493,6 +498,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/join34.q.out b/ql/src/test/results/clientpositive/join34.q.out
index b8fd984..fdeb022 100644
--- a/ql/src/test/results/clientpositive/join34.q.out
+++ b/ql/src/test/results/clientpositive/join34.q.out
@@ -110,6 +110,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 46 Data size: 12236 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 2
                         compressed: false
                         GlobalTableId: 1
 #### A masked pattern was here ####
@@ -153,6 +154,7 @@ STAGE PLANS:
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
+                            bucketingVersion: 1
                             compressed: false
                             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -199,6 +201,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 46 Data size: 12236 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 2
                         compressed: false
                         GlobalTableId: 1
 #### A masked pattern was here ####
@@ -242,6 +245,7 @@ STAGE PLANS:
                           outputColumnNames: _col0, _col1, _col2
                           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
                           File Output Operator
+                            bucketingVersion: 1
                             compressed: false
                             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -408,7 +412,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -450,6 +456,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -460,6 +467,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/join35.q.out b/ql/src/test/results/clientpositive/join35.q.out
index f524ff4..f4d2c0c 100644
--- a/ql/src/test/results/clientpositive/join35.q.out
+++ b/ql/src/test/results/clientpositive/join35.q.out
@@ -75,8 +75,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 74 Data size: 7030 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 74 Data size: 7030 Basic stats: COMPLETE Column stats: COMPLETE
@@ -146,6 +148,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 74 Data size: 7030 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -212,6 +215,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 46 Data size: 8234 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -255,6 +259,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1304 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 1
                         compressed: false
                         GlobalTableId: 0
 #### A masked pattern was here ####
@@ -290,6 +295,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 46 Data size: 8234 Basic stats: COMPLETE Column stats: COMPLETE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -333,6 +339,7 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1304 Basic stats: COMPLETE Column stats: COMPLETE
                       File Output Operator
+                        bucketingVersion: 1
                         compressed: false
                         GlobalTableId: 0
 #### A masked pattern was here ####
@@ -495,7 +502,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 1304 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -537,6 +546,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -547,6 +557,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -579,8 +590,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 74 Data size: 7030 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 74 Data size: 7030 Basic stats: COMPLETE Column stats: COMPLETE
@@ -650,6 +663,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 74 Data size: 7030 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/join9.q.out b/ql/src/test/results/clientpositive/join9.q.out
index 5c11241..0479d64 100644
--- a/ql/src/test/results/clientpositive/join9.q.out
+++ b/ql/src/test/results/clientpositive/join9.q.out
@@ -53,8 +53,10 @@ STAGE PLANS:
                 outputColumnNames: _col0
                 Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -74,8 +76,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: string)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: string)
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
@@ -200,6 +204,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1
             Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 1
 #### A masked pattern was here ####
@@ -243,6 +248,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 864 Basic stats: COMPLETE Column stats: COMPLETE
                 File Output Operator
+                  bucketingVersion: 1
                   compressed: false
                   GlobalTableId: 0
 #### A masked pattern was here ####
@@ -306,7 +312,9 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               null sort order: 
+              numBuckets: -1
               sort order: 
               Statistics: Num rows: 1 Data size: 864 Basic stats: COMPLETE Column stats: COMPLETE
               tag: -1
@@ -348,6 +356,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1
           Statistics: Num rows: 1 Data size: 880 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -358,6 +367,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1
                   columns.types struct<columntype:string,min:bigint,max:bigint,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
diff --git a/ql/src/test/results/clientpositive/join_filters_overlap.q.out b/ql/src/test/results/clientpositive/join_filters_overlap.q.out
index 6bb6de1..90f441c 100644
--- a/ql/src/test/results/clientpositive/join_filters_overlap.q.out
+++ b/ql/src/test/results/clientpositive/join_filters_overlap.q.out
@@ -45,8 +45,10 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: _col0 (type: int)
                 null sort order: z
+                numBuckets: -1
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -67,8 +69,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -144,6 +148,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col4, _col5
           Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -168,8 +173,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
@@ -190,8 +197,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -294,6 +303,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
             Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -304,6 +314,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3,_col4,_col5
                     columns.types int:int:int:int:int:int
                     escape.delim \
@@ -384,8 +395,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -401,8 +414,10 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: _col0 (type: int)
                 null sort order: z
+                numBuckets: -1
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -478,6 +493,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -502,8 +518,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col2 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col2 (type: int)
               Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
@@ -524,8 +542,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -628,6 +648,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
             Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -638,6 +659,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3,_col4,_col5
                     columns.types int:int:int:int:int:int
                     escape.delim \
@@ -718,8 +740,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -735,8 +759,10 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: _col0 (type: int)
                 null sort order: z
+                numBuckets: -1
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -812,6 +838,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -836,8 +863,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col2 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col2 (type: int)
               Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
@@ -858,8 +887,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -962,6 +993,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
             Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -972,6 +1004,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3,_col4,_col5
                     columns.types int:int:int:int:int:int
                     escape.delim \
@@ -1050,8 +1083,10 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: _col0 (type: int)
                 null sort order: z
+                numBuckets: -1
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1067,8 +1102,10 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2, _col3
               Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: _col0 (type: int)
                 null sort order: z
+                numBuckets: -1
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1145,6 +1182,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col6
           Statistics: Num rows: 9 Data size: 216 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1169,8 +1207,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col4 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col4 (type: int)
               Statistics: Num rows: 9 Data size: 216 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1191,8 +1231,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1291,6 +1333,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col4, _col5, _col8, _col9
           Statistics: Num rows: 9 Data size: 252 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1315,8 +1358,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 9 Data size: 252 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1337,8 +1382,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1441,6 +1488,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
             Statistics: Num rows: 9 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1451,6 +1499,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7
                     columns.types int:int:int:int:int:int:int:int
                     escape.delim \
@@ -1521,8 +1570,10 @@ STAGE PLANS:
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
               Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
               Reduce Output Operator
+                bucketingVersion: 2
                 key expressions: _col0 (type: int)
                 null sort order: z
+                numBuckets: -1
                 sort order: +
                 Map-reduce partition columns: _col0 (type: int)
                 Statistics: Num rows: 3 Data size: 60 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1543,8 +1594,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1620,6 +1673,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col3, _col5, _col6
           Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1644,8 +1698,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 3 Data size: 72 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1666,8 +1722,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1766,6 +1824,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2, _col5, _col6, _col7, _col8
           Statistics: Num rows: 3 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
           File Output Operator
+            bucketingVersion: 1
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1790,8 +1849,10 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             Reduce Output Operator
+              bucketingVersion: 2
               key expressions: _col0 (type: int)
               null sort order: z
+              numBuckets: -1
               sort order: +
               Map-reduce partition columns: _col0 (type: int)
               Statistics: Num rows: 3 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1812,8 +1873,10 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1
                 Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                 Reduce Output Operator
+                  bucketingVersion: 2
                   key expressions: _col0 (type: int)
                   null sort order: z
+                  numBuckets: -1
                   sort order: +
                   Map-reduce partition columns: _col0 (type: int)
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1916,6 +1979,7 @@ STAGE PLANS:
             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7
             Statistics: Num rows: 3 Data size: 96 Basic stats: COMPLETE Column stats: COMPLETE
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1926,6 +1990,7 @@ STAGE PLANS:
                   input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                   output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                   properties:
+                    bucketing_version -1
                     columns _col0,_col1,_col2,_col3,_col4,_col5,_col6,_col7
                     columns.types int:int:int:int:int:int:int:int
                     escape.delim \
diff --git a/ql/src/test/results/clientpositive/join_map_ppr.q.out b/ql/src/test/results/clientpositive/join_map_ppr.q.out
index 0e5cc5f..e3d8212 100644
--- a/ql/src/test/results/clientpositive/join_map_ppr.q.out
+++ b/ql/src/test/results/clientpositive/join_map_ppr.q.out
@@ -113,6 +113,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 1100 Data size: 195800 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -156,7 +157,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -226,6 +229,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -236,6 +240,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -302,6 +307,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -392,6 +398,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -765,6 +772,7 @@ STAGE PLANS:
                   outputColumnNames: _col0, _col1, _col2
                   Statistics: Num rows: 1100 Data size: 104500 Basic stats: COMPLETE Column stats: NONE
                   File Output Operator
+                    bucketingVersion: 2
                     compressed: false
                     GlobalTableId: 1
 #### A masked pattern was here ####
@@ -808,7 +816,9 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         null sort order: 
+                        numBuckets: -1
                         sort order: 
                         Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
                         tag: -1
@@ -878,6 +888,7 @@ STAGE PLANS:
           outputColumnNames: _col0, _col1, _col2
           Statistics: Num rows: 1 Data size: 1320 Basic stats: COMPLETE Column stats: NONE
           File Output Operator
+            bucketingVersion: 2
             compressed: false
             GlobalTableId: 0
 #### A masked pattern was here ####
@@ -888,6 +899,7 @@ STAGE PLANS:
                 input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                 output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                 properties:
+                  bucketing_version -1
                   columns _col0,_col1,_col2
                   columns.types struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>:struct<columntype:string,maxlength:bigint,avglength:double,countnulls:bigint,numdistinctvalues:bigint,ndvbitvector:binary>
                   escape.delim \
@@ -954,6 +966,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
@@ -1044,6 +1057,7 @@ STAGE PLANS:
           TableScan
             GatherStats: false
             File Output Operator
+              bucketingVersion: 2
               compressed: false
               GlobalTableId: 0
 #### A masked pattern was here ####
diff --git a/ql/src/test/results/clientpositive/kafka/kafka_storage_handler.q.out b/ql/src/test/results/clientpositive/kafka/kafka_storage_handler.q.out
index 68ea97d..71af39c 100644
--- a/ql/src/test/results/clientpositive/kafka/kafka_storage_handler.q.out
+++ b/ql/src/test/results/clientpositive/kafka/kafka_storage_handler.q.out
@@ -1109,8 +1109,10 @@ STAGE PLANS:
                       outputColumnNames: _col0, _col1, _col2
                       Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                       Reduce Output Operator
+                        bucketingVersion: 2
                         key expressions: _col0 (type: bigint), _col1 (type: timestamp), _col2 (type: binary)
                         null sort order: zzz
+                        numBuckets: -1
                         sort order: +++
                         Map-reduce partition columns: _col0 (type: bigint), _col1 (type: timestamp), _col2 (type: binary)
                         Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
@@ -1334,6 +1336,7 @@ STAGE PLANS:
                 outputColumnNames: _col0, _col1, _col2
                 Statistics: Num rows: 1 Data size: 160 Basic stats: COMPLETE Column stats: NONE
                 File Output Operator
+                  bucketingVersion: 2
                   compressed: false
... 22894 lines suppressed ...