You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mu...@apache.org on 2021/01/14 22:00:37 UTC

[hive] branch master updated: HIVE-24510: Vectorize compute_bit_vector (Mustafa Iman, reviewed by Laszlo Bodor)

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

mustafaiman 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 64cbded  HIVE-24510: Vectorize compute_bit_vector (Mustafa Iman, reviewed by Laszlo Bodor)
64cbded is described below

commit 64cbdedbfbd104df1862b3dbc54f44882bfc0cea
Author: Mustafa Iman <mu...@gmail.com>
AuthorDate: Thu Dec 10 15:07:34 2020 -0800

    HIVE-24510: Vectorize compute_bit_vector (Mustafa Iman, reviewed by Laszlo Bodor)
---
 .../results/clientpositive/serde_typedbytes.q.out  |    4 +-
 .../results/clientpositive/serde_typedbytes2.q.out |    4 +-
 .../results/clientpositive/serde_typedbytes3.q.out |    4 +-
 .../results/clientpositive/serde_typedbytes4.q.out |    2 +-
 .../results/clientpositive/serde_typedbytes5.q.out |    4 +-
 .../hbase_single_sourced_multi_insert.q.out        |    5 +-
 .../src/test/results/clientpositive/explain.q.out  |    2 +-
 .../insert_into_dynamic_partitions.q.out           |    4 +-
 .../results/clientpositive/insert_into_table.q.out |    4 +-
 .../insert_overwrite_dynamic_partitions.q.out      |    4 +-
 .../clientpositive/insert_overwrite_table.q.out    |    4 +-
 .../write_final_output_blobstore.q.out             |    4 +-
 .../UDAFTemplates/VectorUDAFComputeBitVector.txt   |  296 ++++
 .../hadoop/hive/ql/exec/FunctionRegistry.java      |    3 +-
 .../VectorUDAFComputeBitVectorDecimal.java         |  236 +++
 .../VectorUDAFComputeBitVectorString.java          |  228 +++
 .../VectorUDAFComputeBitVectorTimestamp.java       |  234 +++
 .../hive/ql/optimizer/physical/Vectorizer.java     |    1 +
 .../hive/ql/parse/ColumnStatsSemanticAnalyzer.java |   23 +-
 .../udf/generic/GenericUDAFComputeBitVector.java   |  559 ------
 .../generic/GenericUDAFComputeBitVectorBase.java   |  291 +++
 .../GenericUDAFComputeBitVectorFMSketch.java       |  253 +++
 .../generic/GenericUDAFComputeBitVectorHLL.java    |  251 +++
 .../clientpositive/acid_vectorization_original.q   |    8 +
 .../clientpositive/auto_sortmerge_join_14.q        |    3 +
 .../queries/clientpositive/compute_bit_vector.q    |   20 +
 ql/src/test/queries/clientpositive/hll.q           |    1 +
 ql/src/test/queries/clientpositive/insert0.q       |   11 +
 .../queries/clientpositive/vector_data_types.q     |    3 +
 .../test/queries/clientpositive/vectorized_stats.q |   92 +
 .../clientnegative/fileformat_void_input.q.out     |    2 +-
 .../test/results/clientnegative/masking_mv.q.out   |    5 +-
 .../clientpositive/beeline/smb_mapjoin_11.q.out    |    4 +-
 .../clientpositive/beeline/smb_mapjoin_12.q.out    |    4 +-
 .../clientpositive/beeline/smb_mapjoin_7.q.out     |    4 +-
 .../clientpositive/bucket_map_join_spark1.q.out    |    8 +-
 .../clientpositive/bucket_map_join_spark2.q.out    |    8 +-
 .../clientpositive/bucket_map_join_spark3.q.out    |    8 +-
 .../results/clientpositive/bucketmapjoin5.q.out    |    8 +-
 .../clientpositive/bucketmapjoin_negative.q.out    |    5 +-
 .../clientpositive/bucketmapjoin_negative2.q.out   |    5 +-
 .../test/results/clientpositive/join_map_ppr.q.out |   10 +-
 .../results/clientpositive/list_bucket_dml_8.q.out |    5 +-
 .../clientpositive/llap/acid_no_buckets.q.out      |   99 +-
 .../results/clientpositive/llap/acid_stats5.q.out  |    2 +-
 .../clientpositive/llap/autoColumnStats_11.q.out   |   12 +-
 .../clientpositive/llap/autoColumnStats_4.q.out    |    8 +-
 .../clientpositive/llap/autoColumnStats_5.q.out    |   12 +-
 .../clientpositive/llap/autoColumnStats_5a.q.out   |   12 +-
 .../clientpositive/llap/autoColumnStats_6.q.out    |    8 +-
 .../clientpositive/llap/autoColumnStats_7.q.out    |    8 +-
 .../clientpositive/llap/autoColumnStats_8.q.out    |   14 +-
 .../clientpositive/llap/autoColumnStats_9.q.out    |    6 +-
 .../results/clientpositive/llap/auto_join1.q.out   |    8 +-
 .../results/clientpositive/llap/auto_join14.q.out  |    8 +-
 .../results/clientpositive/llap/auto_join17.q.out  |    8 +-
 .../results/clientpositive/llap/auto_join19.q.out  |    8 +-
 .../clientpositive/llap/auto_join19_inclause.q.out |    8 +-
 .../results/clientpositive/llap/auto_join2.q.out   |    8 +-
 .../results/clientpositive/llap/auto_join26.q.out  |    8 +-
 .../results/clientpositive/llap/auto_join3.q.out   |    8 +-
 .../results/clientpositive/llap/auto_join4.q.out   |    8 +-
 .../results/clientpositive/llap/auto_join5.q.out   |    8 +-
 .../results/clientpositive/llap/auto_join6.q.out   |    6 +-
 .../results/clientpositive/llap/auto_join7.q.out   |    6 +-
 .../results/clientpositive/llap/auto_join8.q.out   |    8 +-
 .../results/clientpositive/llap/auto_join9.q.out   |    8 +-
 .../clientpositive/llap/auto_smb_mapjoin_14.q.out  |   26 +-
 .../llap/auto_sortmerge_join_13.q.out              |   42 +-
 .../llap/auto_sortmerge_join_14.q.out              |   30 +-
 .../clientpositive/llap/binary_output_format.q.out |    6 +-
 .../test/results/clientpositive/llap/bucket1.q.out |    8 +-
 .../test/results/clientpositive/llap/bucket2.q.out |    4 +-
 .../test/results/clientpositive/llap/bucket3.q.out |    8 +-
 .../test/results/clientpositive/llap/bucket4.q.out |    4 +-
 .../test/results/clientpositive/llap/bucket5.q.out |   16 +-
 .../test/results/clientpositive/llap/bucket6.q.out |    8 +-
 .../results/clientpositive/llap/bucket_many.q.out  |    8 +-
 .../clientpositive/llap/bucket_num_reducers2.q.out |    2 +-
 .../clientpositive/llap/bucketmapjoin1.q.out       |    8 +-
 .../clientpositive/llap/bucketmapjoin2.q.out       |   12 +-
 .../clientpositive/llap/bucketmapjoin3.q.out       |    8 +-
 .../clientpositive/llap/bucketmapjoin4.q.out       |    8 +-
 .../llap/bucketsortoptimize_insert_1.q.out         |    4 +-
 .../llap/bucketsortoptimize_insert_2.q.out         |   24 +-
 .../llap/bucketsortoptimize_insert_3.q.out         |    4 +-
 .../llap/bucketsortoptimize_insert_4.q.out         |    8 +-
 .../llap/bucketsortoptimize_insert_5.q.out         |    8 +-
 .../llap/bucketsortoptimize_insert_6.q.out         |   28 +-
 .../llap/bucketsortoptimize_insert_7.q.out         |   12 +-
 .../llap/bucketsortoptimize_insert_8.q.out         |    8 +-
 .../clientpositive/llap/case_sensitivity.q.out     |    8 +-
 .../test/results/clientpositive/llap/cast1.q.out   |    8 +-
 .../clientpositive/llap/cbo_rp_auto_join17.q.out   |    8 +-
 .../llap/cbo_rp_gby2_map_multi_distinct.q.out      |   12 +-
 .../cbo_rp_groupby3_noskew_multi_distinct.q.out    |    2 +-
 .../results/clientpositive/llap/clusterctas.q.out  |    8 +-
 .../llap/column_pruner_multiple_children.q.out     |    4 +-
 .../clientpositive/llap/column_table_stats.q.out   |   24 +-
 .../llap/column_table_stats_orc.q.out              |   18 +-
 .../clientpositive/llap/columnstats_partlvl.q.out  |   42 +-
 .../llap/columnstats_partlvl_dp.q.out              |   24 +-
 .../clientpositive/llap/columnstats_quoting.q.out  |   12 +-
 .../clientpositive/llap/columnstats_tbllvl.q.out   |   42 +-
 .../clientpositive/llap/compute_bit_vector.q.out   |  127 ++
 .../clientpositive/llap/compute_stats_date.q.out   |    6 +-
 .../clientpositive/llap/constant_prop_2.q.out      |    6 +-
 .../results/clientpositive/llap/constprog_dp.q.out |    8 +-
 .../clientpositive/llap/constprog_type.q.out       |    8 +-
 .../test/results/clientpositive/llap/cp_sel.q.out  |    8 +-
 ql/src/test/results/clientpositive/llap/ctas.q.out |   40 +-
 .../results/clientpositive/llap/ctas_colname.q.out |   56 +-
 .../llap/ctas_uses_database_location.q.out         |    8 +-
 .../results/clientpositive/llap/cte_mat_7.q.out    |    4 +-
 .../llap/disable_merge_for_bucketing.q.out         |    4 +-
 .../llap/display_colstats_tbllvl.q.out             |   18 +-
 .../clientpositive/llap/dp_counter_mm.q.out        |  212 +--
 .../clientpositive/llap/dp_counter_non_mm.q.out    |  212 +--
 .../llap/dynamic_partition_pruning.q.out           |    4 +-
 .../llap/dynamic_semijoin_reduction_3.q.out        |   24 +-
 .../results/clientpositive/llap/dynpart_cast.q.out |    6 +-
 .../llap/dynpart_sort_opt_bucketing.q.out          |    4 +-
 .../llap/dynpart_sort_opt_vectorization.q.out      |  128 +-
 .../llap/dynpart_sort_optimization.q.out           | 1068 +++++------
 .../llap/dynpart_sort_optimization2.q.out          |   40 +-
 .../llap/dynpart_sort_optimization_acid2.q.out     |    8 +-
 .../llap/enforce_constraint_notnull.q.out          |  220 +--
 .../llap/exec_parallel_column_stats.q.out          |    6 +-
 .../results/clientpositive/llap/explain_ddl.q.out  |   40 +-
 .../clientpositive/llap/explainanalyze_2.q.out     |   24 +-
 .../explainanalyze_acid_with_direct_insert.q.out   |   28 +-
 .../clientpositive/llap/explainuser_1.q.out        |   24 +-
 .../clientpositive/llap/explainuser_2.q.out        |  628 +++----
 .../results/clientpositive/llap/fm-sketch.q.out    |    8 +-
 .../results/clientpositive/llap/groupby10.q.out    |   40 +-
 .../results/clientpositive/llap/groupby11.q.out    |   16 +-
 .../results/clientpositive/llap/groupby12.q.out    |    4 +-
 .../results/clientpositive/llap/groupby1_map.q.out |    8 +-
 .../clientpositive/llap/groupby1_map_nomap.q.out   |    8 +-
 .../clientpositive/llap/groupby1_map_skew.q.out    |    8 +-
 .../clientpositive/llap/groupby1_noskew.q.out      |    4 +-
 .../results/clientpositive/llap/groupby2.q.out     |    8 +-
 .../results/clientpositive/llap/groupby2_map.q.out |    6 +-
 .../llap/groupby2_map_multi_distinct.q.out         |   12 +-
 .../clientpositive/llap/groupby2_map_skew.q.out    |    6 +-
 .../clientpositive/llap/groupby2_noskew.q.out      |    4 +-
 .../llap/groupby2_noskew_multi_distinct.q.out      |    4 +-
 .../results/clientpositive/llap/groupby3.q.out     |    6 +-
 .../results/clientpositive/llap/groupby3_map.q.out |    2 +-
 .../llap/groupby3_map_multi_distinct.q.out         |    2 +-
 .../clientpositive/llap/groupby3_map_skew.q.out    |    2 +-
 .../clientpositive/llap/groupby3_noskew.q.out      |    2 +-
 .../llap/groupby3_noskew_multi_distinct.q.out      |    2 +-
 .../results/clientpositive/llap/groupby4.q.out     |    8 +-
 .../results/clientpositive/llap/groupby4_map.q.out |    4 +-
 .../clientpositive/llap/groupby4_map_skew.q.out    |    4 +-
 .../clientpositive/llap/groupby4_noskew.q.out      |    4 +-
 .../results/clientpositive/llap/groupby5.q.out     |    8 +-
 .../results/clientpositive/llap/groupby5_map.q.out |    4 +-
 .../clientpositive/llap/groupby5_map_skew.q.out    |    4 +-
 .../clientpositive/llap/groupby5_noskew.q.out      |    4 +-
 .../results/clientpositive/llap/groupby6.q.out     |    8 +-
 .../results/clientpositive/llap/groupby6_map.q.out |    8 +-
 .../clientpositive/llap/groupby6_map_skew.q.out    |    8 +-
 .../clientpositive/llap/groupby6_noskew.q.out      |    4 +-
 .../results/clientpositive/llap/groupby7_map.q.out |   16 +-
 .../llap/groupby7_map_multi_single_reducer.q.out   |   12 +-
 .../clientpositive/llap/groupby7_map_skew.q.out    |   16 +-
 .../clientpositive/llap/groupby7_noskew.q.out      |    8 +-
 .../groupby7_noskew_multi_single_reducer.q.out     |    8 +-
 .../results/clientpositive/llap/groupby8.q.out     |   32 +-
 .../results/clientpositive/llap/groupby8_map.q.out |   12 +-
 .../clientpositive/llap/groupby8_map_skew.q.out    |   12 +-
 .../clientpositive/llap/groupby8_noskew.q.out      |    8 +-
 .../results/clientpositive/llap/groupby9.q.out     |   64 +-
 .../clientpositive/llap/groupby_cube1.q.out        |    8 +-
 .../llap/groupby_cube_multi_gby.q.out              |   16 +-
 .../llap/groupby_duplicate_key.q.out               |    4 +-
 .../clientpositive/llap/groupby_map_ppr.q.out      |    6 +-
 .../llap/groupby_map_ppr_multi_distinct.q.out      |    6 +-
 .../groupby_multi_insert_common_distinct.q.out     |   12 +-
 .../llap/groupby_multi_single_reducer2.q.out       |   12 +-
 .../llap/groupby_multi_single_reducer3.q.out       |   48 +-
 .../llap/groupby_nocolumnalign.q.out               |    8 +-
 .../clientpositive/llap/groupby_position.q.out     |   24 +-
 .../results/clientpositive/llap/groupby_ppr.q.out  |    4 +-
 .../llap/groupby_ppr_multi_distinct.q.out          |    8 +-
 .../clientpositive/llap/groupby_rollup1.q.out      |    8 +-
 .../clientpositive/llap/groupby_sort_1_23.q.out    |  166 +-
 .../clientpositive/llap/groupby_sort_2.q.out       |    8 +-
 .../clientpositive/llap/groupby_sort_3.q.out       |   16 +-
 .../clientpositive/llap/groupby_sort_4.q.out       |   16 +-
 .../clientpositive/llap/groupby_sort_5.q.out       |   24 +-
 .../clientpositive/llap/groupby_sort_6.q.out       |   24 +-
 .../clientpositive/llap/groupby_sort_7.q.out       |    8 +-
 .../llap/groupby_sort_skew_1_23.q.out              |  166 +-
 .../clientpositive/llap/groupby_sort_test_1.q.out  |    8 +-
 ql/src/test/results/clientpositive/llap/hll.q.out  |    8 +-
 .../llap/implicit_cast_during_insert.q.out         |    4 +-
 .../llap/infer_bucket_sort_dyn_part.q.out          |    8 +-
 .../infer_bucket_sort_grouping_operators.q.out     |   24 +-
 .../llap/infer_bucket_sort_map_operators.q.out     |   28 +-
 .../llap/infer_bucket_sort_num_buckets.q.out       |    8 +-
 .../results/clientpositive/llap/innerjoin.q.out    |    6 +-
 .../test/results/clientpositive/llap/input11.q.out |    8 +-
 .../test/results/clientpositive/llap/input12.q.out |   20 +-
 .../test/results/clientpositive/llap/input13.q.out |   20 +-
 .../test/results/clientpositive/llap/input14.q.out |    8 +-
 .../test/results/clientpositive/llap/input17.q.out |    8 +-
 .../test/results/clientpositive/llap/input18.q.out |    8 +-
 .../results/clientpositive/llap/input1_limit.q.out |   16 +-
 .../test/results/clientpositive/llap/input20.q.out |    6 +-
 .../test/results/clientpositive/llap/input30.q.out |    8 +-
 .../test/results/clientpositive/llap/input32.q.out |    4 +-
 .../test/results/clientpositive/llap/input33.q.out |    6 +-
 .../test/results/clientpositive/llap/input34.q.out |    6 +-
 .../test/results/clientpositive/llap/input35.q.out |    6 +-
 .../test/results/clientpositive/llap/input36.q.out |    6 +-
 .../test/results/clientpositive/llap/input38.q.out |    6 +-
 .../results/clientpositive/llap/input3_limit.q.out |    8 +-
 .../test/results/clientpositive/llap/input5.q.out  |    8 +-
 .../test/results/clientpositive/llap/input6.q.out  |    8 +-
 .../test/results/clientpositive/llap/input7.q.out  |    8 +-
 .../test/results/clientpositive/llap/input8.q.out  |    8 +-
 .../test/results/clientpositive/llap/input9.q.out  |    8 +-
 .../results/clientpositive/llap/input_part1.q.out  |    8 +-
 .../results/clientpositive/llap/input_part10.q.out |    8 +-
 .../results/clientpositive/llap/input_part2.q.out  |   14 +-
 .../results/clientpositive/llap/input_part5.q.out  |    8 +-
 .../llap/input_testsequencefile.q.out              |    8 +-
 .../clientpositive/llap/input_testxpath.q.out      |    8 +-
 .../clientpositive/llap/input_testxpath2.q.out     |    8 +-
 .../test/results/clientpositive/llap/insert1.q.out |   46 +-
 .../llap/insert1_overwrite_partitions.q.out        |   24 +-
 .../llap/insert2_overwrite_partitions.q.out        |   16 +-
 .../results/clientpositive/llap/insert_into1.q.out |   36 +-
 .../results/clientpositive/llap/insert_into2.q.out |   24 +-
 .../results/clientpositive/llap/insert_into3.q.out |   32 +-
 .../results/clientpositive/llap/insert_into4.q.out |   24 +-
 .../results/clientpositive/llap/insert_into5.q.out |   32 +-
 .../results/clientpositive/llap/insert_into6.q.out |   16 +-
 .../llap/insert_into_default_keyword.q.out         |  146 +-
 .../llap/insert_only_empty_query.q.out             |    8 +-
 .../llap/insert_overwrite_preemption.q.out         |    8 +-
 .../llap/insertoverwrite_bucket.q.out              |   16 +-
 .../test/results/clientpositive/llap/join1.q.out   |    6 +-
 .../test/results/clientpositive/llap/join14.q.out  |    6 +-
 .../test/results/clientpositive/llap/join17.q.out  |    6 +-
 .../test/results/clientpositive/llap/join25.q.out  |    8 +-
 .../test/results/clientpositive/llap/join26.q.out  |    8 +-
 .../test/results/clientpositive/llap/join27.q.out  |    8 +-
 .../test/results/clientpositive/llap/join28.q.out  |    8 +-
 .../test/results/clientpositive/llap/join29.q.out  |    8 +-
 .../test/results/clientpositive/llap/join3.q.out   |    6 +-
 .../test/results/clientpositive/llap/join30.q.out  |    8 +-
 .../test/results/clientpositive/llap/join31.q.out  |    8 +-
 .../test/results/clientpositive/llap/join32.q.out  |    8 +-
 .../clientpositive/llap/join32_lessSize.q.out      |   48 +-
 .../test/results/clientpositive/llap/join33.q.out  |    8 +-
 .../test/results/clientpositive/llap/join34.q.out  |   12 +-
 .../test/results/clientpositive/llap/join35.q.out  |   12 +-
 .../test/results/clientpositive/llap/join36.q.out  |    8 +-
 .../test/results/clientpositive/llap/join37.q.out  |    8 +-
 .../test/results/clientpositive/llap/join39.q.out  |    8 +-
 .../test/results/clientpositive/llap/join4.q.out   |    6 +-
 .../test/results/clientpositive/llap/join5.q.out   |    6 +-
 .../test/results/clientpositive/llap/join6.q.out   |    6 +-
 .../test/results/clientpositive/llap/join7.q.out   |    6 +-
 .../test/results/clientpositive/llap/join8.q.out   |    6 +-
 .../test/results/clientpositive/llap/join9.q.out   |    6 +-
 .../llap/limit_pushdown_negative.q.out             |   14 +-
 .../results/clientpositive/llap/lineage1.q.out     |    6 +-
 .../clientpositive/llap/list_bucket_dml_1.q.out    |    8 +-
 .../clientpositive/llap/list_bucket_dml_10.q.out   |    8 +-
 .../clientpositive/llap/list_bucket_dml_11.q.out   |    8 +-
 .../clientpositive/llap/list_bucket_dml_12.q.out   |    8 +-
 .../clientpositive/llap/list_bucket_dml_13.q.out   |    8 +-
 .../clientpositive/llap/list_bucket_dml_14.q.out   |    8 +-
 .../clientpositive/llap/list_bucket_dml_2.q.out    |    8 +-
 .../clientpositive/llap/list_bucket_dml_3.q.out    |    8 +-
 .../clientpositive/llap/list_bucket_dml_4.q.out    |   16 +-
 .../clientpositive/llap/list_bucket_dml_5.q.out    |    8 +-
 .../clientpositive/llap/list_bucket_dml_6.q.out    |   16 +-
 .../clientpositive/llap/list_bucket_dml_7.q.out    |   16 +-
 .../clientpositive/llap/list_bucket_dml_9.q.out    |   16 +-
 .../results/clientpositive/llap/llap_stats.q.out   |    6 +-
 .../clientpositive/llap/load_dyn_part1.q.out       |   14 +-
 .../clientpositive/llap/load_dyn_part10.q.out      |    8 +-
 .../clientpositive/llap/load_dyn_part13.q.out      |   12 +-
 .../clientpositive/llap/load_dyn_part14.q.out      |   16 +-
 .../clientpositive/llap/load_dyn_part2.q.out       |    8 +-
 .../clientpositive/llap/load_dyn_part4.q.out       |    8 +-
 .../clientpositive/llap/load_dyn_part5.q.out       |    8 +-
 .../clientpositive/llap/load_dyn_part8.q.out       |   14 +-
 .../clientpositive/llap/load_dyn_part9.q.out       |    8 +-
 .../llap/mapjoin_decimal_vectorized.q.out          |    4 +-
 .../results/clientpositive/llap/mapreduce1.q.out   |    8 +-
 .../results/clientpositive/llap/mapreduce2.q.out   |    8 +-
 .../results/clientpositive/llap/mapreduce3.q.out   |    8 +-
 .../results/clientpositive/llap/mapreduce4.q.out   |    8 +-
 .../results/clientpositive/llap/mapreduce5.q.out   |    8 +-
 .../results/clientpositive/llap/mapreduce6.q.out   |    8 +-
 .../results/clientpositive/llap/mapreduce7.q.out   |    8 +-
 .../results/clientpositive/llap/mapreduce8.q.out   |    8 +-
 .../results/clientpositive/llap/masking_mv.q.out   |   16 +-
 .../clientpositive/llap/masking_mv_by_text.q.out   |    8 +-
 .../llap/materialized_view_cluster.q.out           |   38 +-
 .../llap/materialized_view_create_rewrite_3.q.out  |    8 +-
 .../llap/materialized_view_create_rewrite_4.q.out  |   20 +-
 .../llap/materialized_view_create_rewrite_5.q.out  |   16 +-
 ...ialized_view_create_rewrite_rebuild_dummy.q.out |   16 +-
 ...erialized_view_create_rewrite_time_window.q.out |   16 +-
 .../llap/materialized_view_distribute_sort.q.out   |   38 +-
 .../llap/materialized_view_partition_cluster.q.out |   38 +-
 .../llap/materialized_view_partitioned.q.out       |   22 +-
 .../llap/materialized_view_partitioned_3.q.out     |    8 +-
 .../llap/materialized_view_rebuild_2.q.out         |    8 +-
 .../llap/materialized_view_rebuild_3.q.out         |    8 +-
 .../llap/materialized_view_rewrite_window.q.out    |   24 +-
 .../test/results/clientpositive/llap/merge1.q.out  |   24 +-
 .../test/results/clientpositive/llap/merge2.q.out  |   24 +-
 .../test/results/clientpositive/llap/merge3.q.out  |   20 +-
 .../test/results/clientpositive/llap/merge4.q.out  |   24 +-
 .../llap/merge_dynamic_partition.q.out             |   12 +-
 .../llap/merge_dynamic_partition2.q.out            |    4 +-
 .../llap/merge_dynamic_partition3.q.out            |    4 +-
 .../llap/merge_dynamic_partition4.q.out            |    4 +-
 .../llap/merge_dynamic_partition5.q.out            |    4 +-
 .../test/results/clientpositive/llap/mm_all.q.out  |    8 +-
 .../test/results/clientpositive/llap/mm_dp.q.out   |    8 +-
 .../results/clientpositive/llap/multi_insert.q.out |  184 +-
 .../clientpositive/llap/multi_insert_gby.q.out     |   24 +-
 .../clientpositive/llap/multi_insert_gby2.q.out    |    4 +-
 .../clientpositive/llap/multi_insert_gby3.q.out    |   60 +-
 .../clientpositive/llap/multi_insert_gby4.q.out    |   18 +-
 .../llap/multi_insert_lateral_view.q.out           |   94 +-
 .../clientpositive/llap/multi_insert_mixed.q.out   |   16 +-
 ...ulti_insert_move_tasks_share_dependencies.q.out |  240 +--
 .../llap/multi_insert_union_src.q.out              |    8 +-
 .../llap/multi_insert_with_join2.q.out             |   84 +-
 .../llap/multigroupby_singlemr.q.out               |   84 +-
 .../llap/nonreserved_keywords_insert_into1.q.out   |   24 +-
 .../clientpositive/llap/notable_alias1.q.out       |    8 +-
 .../clientpositive/llap/notable_alias2.q.out       |    8 +-
 .../clientpositive/llap/nullformatCTAS.q.out       |    8 +-
 .../clientpositive/llap/orc_createas1.q.out        |    8 +-
 .../results/clientpositive/llap/orc_merge1.q.out   |   12 +-
 .../results/clientpositive/llap/orc_merge10.q.out  |   12 +-
 .../results/clientpositive/llap/orc_merge2.q.out   |    4 +-
 .../results/clientpositive/llap/orc_merge3.q.out   |    4 +-
 .../results/clientpositive/llap/orc_merge4.q.out   |    4 +-
 .../results/clientpositive/llap/orc_merge5.q.out   |    8 +-
 .../results/clientpositive/llap/orc_merge6.q.out   |    8 +-
 .../results/clientpositive/llap/orc_merge7.q.out   |    8 +-
 .../clientpositive/llap/orc_merge_diff_fs.q.out    |   12 +-
 .../clientpositive/llap/orc_merge_incompat1.q.out  |    4 +-
 .../clientpositive/llap/orc_merge_incompat2.q.out  |    4 +-
 .../results/clientpositive/llap/parallel.q.out     |   12 +-
 .../clientpositive/llap/parallel_colstats.q.out    |   12 +-
 .../clientpositive/llap/parallel_join1.q.out       |    6 +-
 .../clientpositive/llap/parallel_orderby.q.out     |    4 +-
 .../clientpositive/llap/partial_column_stats.q.out |    6 +-
 .../clientpositive/llap/partition_ctas.q.out       |    8 +-
 ql/src/test/results/clientpositive/llap/pcr.q.out  |   28 +-
 .../clientpositive/llap/ppd_constant_expr.q.out    |   16 +-
 .../clientpositive/llap/ppd_multi_insert.q.out     |   36 +-
 ql/src/test/results/clientpositive/llap/ptf.q.out  |   14 +-
 .../test/results/clientpositive/llap/quote1.q.out  |    8 +-
 .../llap/rand_partitionpruner2.q.out               |    8 +-
 .../clientpositive/llap/rcfile_createas1.q.out     |    8 +-
 .../clientpositive/llap/rcfile_merge2.q.out        |    8 +-
 .../clientpositive/llap/rcfile_merge3.q.out        |    8 +-
 .../clientpositive/llap/rcfile_merge4.q.out        |    8 +-
 .../clientpositive/llap/rcfile_null_value.q.out    |    6 +-
 .../clientpositive/llap/reduce_deduplicate.q.out   |   10 +-
 .../test/results/clientpositive/llap/sample1.q.out |    8 +-
 .../test/results/clientpositive/llap/sample5.q.out |    8 +-
 .../test/results/clientpositive/llap/sample6.q.out |    8 +-
 .../test/results/clientpositive/llap/sample7.q.out |    8 +-
 .../clientpositive/llap/semijoin_hint.q.out        |   16 +-
 .../clientpositive/llap/semijoin_reddedup.q.out    |    8 +-
 .../clientpositive/llap/show_functions.q.out       |    9 +-
 .../sketches_materialized_view_cume_dist.q.out     |    6 +-
 .../llap/sketches_materialized_view_ntile.q.out    |    6 +-
 ...ketches_materialized_view_percentile_disc.q.out |    6 +-
 .../llap/sketches_materialized_view_rank.q.out     |    6 +-
 .../llap/sketches_materialized_view_rollup2.q.out  |    6 +-
 .../llap/sketches_materialized_view_safety.q.out   |   16 +-
 .../results/clientpositive/llap/skewjoin.q.out     |    4 +-
 .../clientpositive/llap/skewjoin_noskew.q.out      |    8 +-
 .../clientpositive/llap/skewjoin_onesideskew.q.out |    6 +-
 .../clientpositive/llap/smb_mapjoin_18.q.out       |    8 +-
 .../clientpositive/llap/smb_mapjoin_6.q.out        |   24 +-
 .../results/clientpositive/llap/sqlmerge.q.out     |   16 +-
 .../clientpositive/llap/sqlmerge_stats.q.out       |   56 +-
 .../test/results/clientpositive/llap/stats0.q.out  |   32 +-
 .../test/results/clientpositive/llap/stats1.q.out  |   12 +-
 .../test/results/clientpositive/llap/stats10.q.out |    8 +-
 .../test/results/clientpositive/llap/stats11.q.out |   12 +-
 .../test/results/clientpositive/llap/stats4.q.out  |   14 +-
 .../clientpositive/llap/stats_empty_dyn_part.q.out |    8 +-
 .../clientpositive/llap/stats_nonpart.q.out        |    8 +-
 .../clientpositive/llap/stats_sizebug.q.out        |    8 +-
 .../clientpositive/llap/subquery_multiinsert.q.out |   22 +-
 .../results/clientpositive/llap/tablevalues.q.out  |    6 +-
 .../llap/temp_table_display_colstats_tbllvl.q.out  |   18 +-
 .../temp_table_insert1_overwrite_partitions.q.out  |   16 +-
 .../temp_table_insert2_overwrite_partitions.q.out  |   16 +-
 .../llap/temp_table_merge_dynamic_partition.q.out  |    4 +-
 .../clientpositive/llap/tez_compile_counters.q.out |    6 +-
 .../test/results/clientpositive/llap/tez_dml.q.out |   26 +-
 .../clientpositive/llap/tez_input_counters.q.out   |    6 +-
 .../llap/tez_union_dynamic_partition.q.out         |    6 +-
 .../llap/tez_union_dynamic_partition_2.q.out       |    6 +-
 .../llap/tez_union_multiinsert.q.out               |   60 +-
 ql/src/test/results/clientpositive/llap/udf1.q.out |    8 +-
 ql/src/test/results/clientpositive/llap/udf3.q.out |    4 +-
 .../results/clientpositive/llap/udf_10_trims.q.out |    8 +-
 .../clientpositive/llap/udf_character_length.q.out |    4 +-
 .../results/clientpositive/llap/udf_length.q.out   |    8 +-
 .../clientpositive/llap/udf_octet_length.q.out     |    4 +-
 .../results/clientpositive/llap/udf_reverse.q.out  |    8 +-
 .../test/results/clientpositive/llap/union10.q.out |   16 +-
 .../test/results/clientpositive/llap/union12.q.out |   16 +-
 .../test/results/clientpositive/llap/union17.q.out |   12 +-
 .../test/results/clientpositive/llap/union18.q.out |   20 +-
 .../test/results/clientpositive/llap/union19.q.out |   20 +-
 .../test/results/clientpositive/llap/union22.q.out |   12 +-
 .../test/results/clientpositive/llap/union25.q.out |    8 +-
 .../test/results/clientpositive/llap/union28.q.out |   16 +-
 .../test/results/clientpositive/llap/union29.q.out |   16 +-
 .../test/results/clientpositive/llap/union30.q.out |   20 +-
 .../test/results/clientpositive/llap/union31.q.out |   40 +-
 .../test/results/clientpositive/llap/union33.q.out |   24 +-
 .../test/results/clientpositive/llap/union4.q.out  |   12 +-
 .../test/results/clientpositive/llap/union6.q.out  |   12 +-
 .../clientpositive/llap/unionDistinct_1.q.out      |  152 +-
 .../clientpositive/llap/unionDistinct_3.q.out      |    8 +-
 .../clientpositive/llap/union_lateralview.q.out    |    6 +-
 .../clientpositive/llap/union_top_level.q.out      |   48 +-
 .../llap/vector_char_varchar_1.q.out               |  150 +-
 .../clientpositive/llap/vector_decimal_6.q.out     |   42 +-
 .../clientpositive/llap/vector_groupby4.q.out      |   44 +-
 .../clientpositive/llap/vector_groupby6.q.out      |   44 +-
 .../clientpositive/llap/vector_groupby_cube1.q.out |  156 +-
 .../clientpositive/llap/vector_windowing.q.out     |  154 +-
 .../llap/vector_windowing_streaming.q.out          |   91 +-
 .../vectorized_dynamic_partition_pruning.q.out     |   18 +-
 .../clientpositive/llap/vectorized_ptf.q.out       |  121 +-
 .../clientpositive/llap/vectorized_stats.q.out     | 1862 ++++++++++++++++++++
 .../test/results/clientpositive/nonmr_fetch.q.out  |    4 +-
 .../test/results/clientpositive/smb_mapjoin9.q.out |    4 +-
 .../results/clientpositive/smb_mapjoin_20.q.out    |   12 +-
 .../results/clientpositive/smb_mapjoin_21.q.out    |   20 +-
 .../clientpositive/tez/explainanalyze_1.q.out      |    4 +-
 .../clientpositive/tez/explainanalyze_3.q.out      |   30 +-
 .../clientpositive/tez/explainanalyze_5.q.out      |    4 +-
 .../results/clientpositive/tez/explainuser_3.q.out |   66 +-
 .../clientpositive/tez/tez_union_udtf.q.out        |   34 +-
 .../hive/common/ndv/NumDistinctValueEstimator.java |    2 +
 .../apache/hadoop/hive/common/ndv/fm/FMSketch.java |    5 +
 .../hadoop/hive/common/ndv/hll/HyperLogLog.java    |    9 +
 .../hive/common/ndv/hll/HyperLogLogUtils.java      |   19 +-
 .../apache/hadoop/hive/tools/GenVectorCode.java    |   29 +-
 464 files changed, 9205 insertions(+), 5046 deletions(-)

diff --git a/contrib/src/test/results/clientpositive/serde_typedbytes.q.out b/contrib/src/test/results/clientpositive/serde_typedbytes.q.out
index c9f70b8..5f71071 100644
--- a/contrib/src/test/results/clientpositive/serde_typedbytes.q.out
+++ b/contrib/src/test/results/clientpositive/serde_typedbytes.q.out
@@ -81,7 +81,7 @@ STAGE PLANS:
                     outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -93,7 +93,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
           Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out b/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out
index 74b1c6c..343462b 100644
--- a/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out
+++ b/contrib/src/test/results/clientpositive/serde_typedbytes2.q.out
@@ -77,7 +77,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -89,7 +89,7 @@ STAGE PLANS:
                       value expressions: _col0 (type: smallint), _col1 (type: smallint), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
           Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out b/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out
index 94b6566..10760ed 100644
--- a/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out
+++ b/contrib/src/test/results/clientpositive/serde_typedbytes3.q.out
@@ -77,7 +77,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -89,7 +89,7 @@ STAGE PLANS:
                       value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
           Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/contrib/src/test/results/clientpositive/serde_typedbytes4.q.out b/contrib/src/test/results/clientpositive/serde_typedbytes4.q.out
index 5bd0e3c..7d350ff 100644
--- a/contrib/src/test/results/clientpositive/serde_typedbytes4.q.out
+++ b/contrib/src/test/results/clientpositive/serde_typedbytes4.q.out
@@ -88,7 +88,7 @@ STAGE PLANS:
             outputColumnNames: key, value
             Statistics: Num rows: 166 Data size: 15770 Basic stats: COMPLETE Column stats: COMPLETE
             Group By Operator
-              aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+              aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
               mode: complete
               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
               Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out b/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out
index 5c4c981..9cf661f 100644
--- a/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out
+++ b/contrib/src/test/results/clientpositive/serde_typedbytes5.q.out
@@ -81,7 +81,7 @@ STAGE PLANS:
                     outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -93,7 +93,7 @@ STAGE PLANS:
                         value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
           Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out b/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out
index a90bb5f..5554bd3 100644
--- a/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out
+++ b/hbase-handler/src/test/results/positive/hbase_single_sourced_multi_insert.q.out
@@ -96,7 +96,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 55 Data size: 9405 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -121,9 +121,10 @@ STAGE PLANS:
                       output format: org.apache.hadoop.hive.hbase.HiveHBaseTableOutputFormat
                       serde: org.apache.hadoop.hive.hbase.HBaseSerDe
                       name: default.src_x2
+      Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
           Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/itests/hive-blobstore/src/test/results/clientpositive/explain.q.out b/itests/hive-blobstore/src/test/results/clientpositive/explain.q.out
index 97b9fb7..258a6b2 100644
--- a/itests/hive-blobstore/src/test/results/clientpositive/explain.q.out
+++ b/itests/hive-blobstore/src/test/results/clientpositive/explain.q.out
@@ -88,7 +88,7 @@ STAGE PLANS:
               outputColumnNames: cnt
               Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
               Group By Operator
-                aggregations: min(cnt), max(cnt), count(1), count(cnt), compute_bit_vector(cnt, 'hll')
+                aggregations: min(cnt), max(cnt), count(1), count(cnt), compute_bit_vector_hll(cnt)
                 mode: complete
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
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 a839649..2e188c9 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
@@ -125,7 +125,7 @@ STAGE PLANS:
                     outputColumnNames: id, key
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(id), max(id), count(1), count(id), compute_bit_vector(id, 'hll')
+                      aggregations: min(id), max(id), count(1), count(id), compute_bit_vector_hll(id)
                       keys: key (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -310,7 +310,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
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 51b5120..062ad8b 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
@@ -110,7 +110,7 @@ STAGE PLANS:
                     outputColumnNames: id
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(id), max(id), count(1), count(id), compute_bit_vector(id, 'hll')
+                      aggregations: min(id), max(id), count(1), count(id), compute_bit_vector_hll(id)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
@@ -164,7 +164,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
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 b3bf13c..810c547 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
@@ -143,7 +143,7 @@ STAGE PLANS:
                     outputColumnNames: id, key
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(id), max(id), count(1), count(id), compute_bit_vector(id, 'hll')
+                      aggregations: min(id), max(id), count(1), count(id), compute_bit_vector_hll(id)
                       keys: key (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -328,7 +328,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
           keys: KEY._col0 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
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 502fdd2..7aef883 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
@@ -118,7 +118,7 @@ STAGE PLANS:
                     outputColumnNames: id
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(id), max(id), count(1), count(id), compute_bit_vector(id, 'hll')
+                      aggregations: min(id), max(id), count(1), count(id), compute_bit_vector_hll(id)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
@@ -172,7 +172,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
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 7214dcd..6ce72d6 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
@@ -215,7 +215,7 @@ STAGE PLANS:
             outputColumnNames: key
             Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
-              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll')
+              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key)
               mode: complete
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
               Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE
@@ -471,7 +471,7 @@ STAGE PLANS:
             outputColumnNames: key
             Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
-              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll')
+              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key)
               mode: complete
               outputColumnNames: _col0, _col1, _col2, _col3, _col4
               Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: NONE
diff --git a/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFComputeBitVector.txt b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFComputeBitVector.txt
new file mode 100644
index 0000000..e48c921
--- /dev/null
+++ b/ql/src/gen/vectorization/UDAFTemplates/VectorUDAFComputeBitVector.txt
@@ -0,0 +1,296 @@
+/*
+ * 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.exec.vector.expressions.aggregates;
+
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationDesc;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.common.ndv.hll.HyperLogLogUtils;
+import org.apache.hadoop.hive.ql.exec.vector.LongColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.DoubleColumnVector;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+
+/**
+ * Generated from template VectorUDAFComputeBitVector.txt.
+ */
+@Description(name = "compute_bit_vector_hll",
+    value = "_FUNC_(x) - Computes bit vector for NDV computation")
+public class <ClassName> extends VectorAggregateExpression {
+
+  public <ClassName>() {
+    super();
+  }
+
+  public <ClassName>(VectorAggregationDesc vecAggrDesc) {
+    super(vecAggrDesc);
+  }
+
+  @Override
+  public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+    return new Aggregation();
+  }
+
+  @Override
+  public void aggregateInput(AggregationBuffer agg, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+#IF COMPLETE
+    <InputColumnVectorType> inputColumn = (<InputColumnVectorType>) batch.cols[this.inputExpression.getOutputColumnNum()];
+#ENDIF COMPLETE
+#IF MERGING
+    BytesColumnVector inputColumn = (BytesColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+#ENDIF MERGING
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+    Aggregation myagg = (Aggregation) agg;
+
+#IF COMPLETE
+    myagg.prepare();
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        myagg.estimator.addToEstimator(inputColumn.vector[0]);
+      } else {
+        if (batch.selectedInUse) {
+          for (int i = 0; i < batchSize; i++) {
+            int s = batch.selected[i];
+            myagg.estimator.addToEstimator(inputColumn.vector[s]);
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            myagg.estimator.addToEstimator(inputColumn.vector[i]);
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          myagg.estimator.addToEstimator(inputColumn.vector[0]);
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int i = 0; i < batchSize; i++) {
+            int s = batch.selected[i];
+            if (!inputColumn.isNull[s]) {
+              myagg.estimator.addToEstimator(inputColumn.vector[s]);
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.vector[i]);
+            }
+          }
+        }
+      }
+    }
+#ENDIF COMPLETE
+#IF MERGING
+    if (inputColumn.isRepeating) {
+      if (!inputColumn.isNull[0] && inputColumn.length[0] > 0) {
+        myagg.prepare();
+        HyperLogLog mergingHLL = HyperLogLogUtils.deserializeHLL(inputColumn.vector[0], inputColumn.start[0], inputColumn.length[0]);
+        myagg.estimator.mergeEstimators(mergingHLL);
+      }
+    } else {
+      for (int i = 0; i < batchSize; i++) {
+        int s = i;
+        if (batch.selectedInUse) {
+          s = batch.selected[i];
+        }
+        if (!inputColumn.isNull[s] && inputColumn.length[s] > 0) {
+          myagg.prepare();
+          HyperLogLog mergingHLL = HyperLogLogUtils.deserializeHLL(inputColumn.vector[s], inputColumn.start[s], inputColumn.length[s]);
+          myagg.estimator.mergeEstimators(mergingHLL);
+        }
+      }
+    }
+#ENDIF MERGING
+  }
+
+  private Aggregation getAggregation(VectorAggregationBufferRow[] sets, int rowid, int bufferIndex) {
+    VectorAggregationBufferRow bufferRow = sets[rowid];
+    Aggregation myagg = (Aggregation) bufferRow.getAggregationBuffer(bufferIndex);
+    myagg.prepare();
+    return myagg;
+  }
+
+  @Override
+  public void aggregateInputSelection(VectorAggregationBufferRow[] aggregationBufferSets, int aggregateIndex, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+#IF COMPLETE
+    <InputColumnVectorType> inputColumn = (<InputColumnVectorType>) batch.cols[this.inputExpression.getOutputColumnNum()];
+#ENDIF COMPLETE
+#IF MERGING
+    BytesColumnVector inputColumn = (BytesColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+#ENDIF MERGING
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+#IF COMPLETE
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        for (int i = 0; i < batchSize; i++) {
+          Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+          myagg.estimator.addToEstimator(inputColumn.vector[0]);
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int i = 0; i < batchSize; i++) {
+            int s = batch.selected[i];
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[s]);
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[i]);
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          for (int i = 0; i < batchSize; i++) {
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[0]);
+          }
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int i = 0; i < batchSize; i++) {
+            int s = batch.selected[i];
+            if (!inputColumn.isNull[s]) {
+              Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+              myagg.estimator.addToEstimator(inputColumn.vector[s]);
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+              myagg.estimator.addToEstimator(inputColumn.vector[i]);
+            }
+          }
+        }
+      }
+    }
+#ENDIF COMPLETE
+#IF MERGING
+    if (inputColumn.isRepeating) {
+      if (!inputColumn.isNull[0] && inputColumn.length[0] > 0) {
+        for (int i = 0; i < batchSize; i++) {
+          Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+          HyperLogLog mergingHLL = HyperLogLogUtils.deserializeHLL(inputColumn.vector[0], inputColumn.start[0], inputColumn.length[0]);
+          myagg.estimator.mergeEstimators(mergingHLL);
+        }
+      }
+    } else {
+      for (int i = 0; i < batchSize; i++) {
+        int s = i;
+        if (batch.selectedInUse) {
+          s = batch.selected[i];
+        }
+        if (!inputColumn.isNull[s] && inputColumn.length[s] > 0) {
+          Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+          HyperLogLog mergingHLL = HyperLogLogUtils.deserializeHLL(inputColumn.vector[s], inputColumn.start[s], inputColumn.length[s]);
+          myagg.estimator.mergeEstimators(mergingHLL);
+        }
+      }
+    }
+#ENDIF MERGING
+  }
+
+  @Override
+  public void reset(AggregationBuffer agg) throws HiveException {
+    agg.reset();
+  }
+
+  @Override
+  public long getAggregationBufferFixedSize() {
+    return Aggregation.FIXED_SIZE;
+  }
+
+  @Override
+  public boolean matches(String name, ColumnVector.Type inputColVectorType, ColumnVector.Type outputColVectorType, GenericUDAFEvaluator.Mode mode) {
+    return name.equals("compute_bit_vector_hll") &&
+        outputColVectorType == ColumnVector.Type.BYTES &&
+#IF MERGING
+        inputColVectorType == ColumnVector.Type.BYTES &&
+        (mode == GenericUDAFEvaluator.Mode.PARTIAL2 || mode == GenericUDAFEvaluator.Mode.FINAL);
+#ENDIF MERGING
+#IF COMPLETE
+        inputColVectorType == ColumnVector.Type.<UpperCaseColumnVectorType> &&
+        (mode == GenericUDAFEvaluator.Mode.PARTIAL1 || mode == GenericUDAFEvaluator.Mode.COMPLETE);
+#ENDIF COMPLETE
+  }
+
+  @Override
+  public void assignRowColumn(VectorizedRowBatch batch, int batchIndex, int columnNum, AggregationBuffer agg) throws HiveException {
+    Aggregation myagg = (Aggregation) agg;
+    BytesColumnVector outputCol = (BytesColumnVector) batch.cols[columnNum];
+    if (myagg.estimator == null) {
+      outputCol.isNull[batchIndex] = true;
+      outputCol.noNulls = false;
+    } else {
+      outputCol.isNull[batchIndex] = false;
+      outputCol.isRepeating = false;
+      byte[] outputbuf = myagg.estimator.serialize();
+      outputCol.setRef(batchIndex, outputbuf, 0, outputbuf.length);
+    }
+  }
+
+  static class Aggregation implements AggregationBuffer {
+
+    private static final long FIXED_SIZE = HyperLogLog.builder().setSizeOptimized().build().lengthFor(null);
+
+    HyperLogLog estimator;
+
+    @Override
+    public int getVariableSize() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void reset() {
+      estimator = null;
+    }
+
+    public void prepare() {
+      if (estimator == null) {
+        estimator = HyperLogLog.builder().setSizeOptimized().build();
+      }
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
index 45ffb8a..994803e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/FunctionRegistry.java
@@ -469,7 +469,8 @@ public final class FunctionRegistry {
 
     system.registerGenericUDAF("compute_stats", new GenericUDAFComputeStats());
     system.registerGenericUDF("ndv_compute_bit_vector", GenericUDFNDVComputeBitVector.class);
-    system.registerGenericUDAF("compute_bit_vector", new GenericUDAFComputeBitVector());
+    system.registerGenericUDAF("compute_bit_vector_hll", new GenericUDAFComputeBitVectorHLL());
+    system.registerGenericUDAF("compute_bit_vector_fm", new GenericUDAFComputeBitVectorFMSketch());
     system.registerGenericUDAF("bloom_filter", new GenericUDAFBloomFilter());
     system.registerGenericUDAF("approx_distinct", new GenericUDAFApproximateDistinct());
     system.registerUDAF("percentile", UDAFPercentile.class);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorDecimal.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorDecimal.java
new file mode 100644
index 0000000..f380efe
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorDecimal.java
@@ -0,0 +1,236 @@
+/*
+ * 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.exec.vector.expressions.aggregates;
+
+import org.apache.hadoop.hive.common.ndv.NumDistinctValueEstimator;
+import org.apache.hadoop.hive.ql.exec.vector.DecimalColumnVector;
+import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationDesc;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+
+/**
+ * Generated from template VectorUDAFComputeBitVector.txt.
+ */
+
+@Description(name = "compute_bit_vector_hll",
+    value = "_FUNC_(x) - Computes bit vector for NDV computation")
+public class VectorUDAFComputeBitVectorDecimal extends VectorAggregateExpression {
+
+  public VectorUDAFComputeBitVectorDecimal() {
+    super();
+  }
+
+  public VectorUDAFComputeBitVectorDecimal(VectorAggregationDesc vecAggrDesc) {
+    super(vecAggrDesc);
+  }
+
+  @Override
+  public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+    return new Aggregation();
+  }
+
+  @Override
+  public void aggregateInput(AggregationBuffer agg, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+    DecimalColumnVector inputColumn = (DecimalColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+    Aggregation myagg = (Aggregation) agg;
+
+    myagg.prepare();
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        myagg.estimator.addToEstimator(inputColumn.vector[0].doubleValue());
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            myagg.estimator.addToEstimator(inputColumn.vector[i].doubleValue());
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            myagg.estimator.addToEstimator(inputColumn.vector[i].doubleValue());
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          myagg.estimator.addToEstimator(inputColumn.vector[0].doubleValue());
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int j = 0; j < batchSize; ++j) {
+            int i = batch.selected[j];
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.vector[i].doubleValue());
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.vector[i].doubleValue());
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private Aggregation getAggregation(VectorAggregationBufferRow[] sets, int rowid, int bufferIndex) {
+    VectorAggregationBufferRow bufferRow = sets[rowid];
+    Aggregation myagg = (Aggregation) bufferRow.getAggregationBuffer(bufferIndex);
+    myagg.prepare();
+    return myagg;
+  }
+
+  @Override
+  public void aggregateInputSelection(VectorAggregationBufferRow[] aggregationBufferSets, int aggregateIndex, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+    DecimalColumnVector inputColumn = (DecimalColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        for (int i = 0; i < batchSize; i++) {
+          Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+          myagg.estimator.addToEstimator(inputColumn.vector[0].doubleValue());
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            Aggregation myagg = getAggregation(aggregationBufferSets, s, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[i].doubleValue());
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[i].doubleValue());
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          for (int i = 0; i < batchSize; i++) {
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[0].doubleValue());
+          }
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            if (!inputColumn.isNull[i]) {
+              Aggregation myagg = getAggregation(aggregationBufferSets, s, aggregateIndex);
+              myagg.estimator.addToEstimator(inputColumn.vector[i].doubleValue());
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+              myagg.estimator.addToEstimator(inputColumn.vector[i].doubleValue());
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void reset(AggregationBuffer agg) throws HiveException {
+    agg.reset();
+  }
+
+  @Override
+  public long getAggregationBufferFixedSize() {
+    return Aggregation.FIXED_SIZE;
+  }
+
+  @Override
+  public boolean matches(String name, ColumnVector.Type inputColVectorType, ColumnVector.Type outputColVectorType, GenericUDAFEvaluator.Mode mode) {
+    return name.equals("compute_bit_vector_hll") &&
+        outputColVectorType == ColumnVector.Type.BYTES &&
+        inputColVectorType == ColumnVector.Type.DECIMAL &&
+        (mode == GenericUDAFEvaluator.Mode.PARTIAL1 || mode == GenericUDAFEvaluator.Mode.COMPLETE);
+  }
+
+  @Override
+  public void assignRowColumn(VectorizedRowBatch batch, int batchIndex, int columnNum, AggregationBuffer agg) throws HiveException {
+    Aggregation myagg = (Aggregation) agg;
+    BytesColumnVector outputCol = (BytesColumnVector) batch.cols[columnNum];
+    if (myagg.estimator == null) {
+      outputCol.isNull[batchIndex] = true;
+      outputCol.noNulls = false;
+    } else {
+      outputCol.isNull[batchIndex] = false;
+      outputCol.isRepeating = false;
+      byte[] outputbuf = myagg.estimator.serialize();
+      outputCol.setRef(batchIndex, outputbuf, 0, outputbuf.length);
+    }
+  }
+
+  static class Aggregation implements AggregationBuffer {
+
+    private static final long FIXED_SIZE = HyperLogLog.builder().setSizeOptimized().build().lengthFor(null);
+
+    HyperLogLog estimator;
+
+    public Aggregation() {
+      reset();
+    }
+
+    @Override
+    public int getVariableSize() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void reset() {
+      estimator = null;
+    }
+
+    public void prepare() {
+      if (estimator == null) {
+        estimator = HyperLogLog.builder().setSizeOptimized().build();
+      }
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorString.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorString.java
new file mode 100644
index 0000000..0a8b82d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorString.java
@@ -0,0 +1,228 @@
+/*
+ * 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.exec.vector.expressions.aggregates;
+
+import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationDesc;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+
+@Description(name = "compute_bit_vector_hll",
+    value = "_FUNC_(x) - Computes bit vector for NDV computation")
+public class VectorUDAFComputeBitVectorString extends VectorAggregateExpression {
+
+  public VectorUDAFComputeBitVectorString() {
+    super();
+  }
+
+  public VectorUDAFComputeBitVectorString(VectorAggregationDesc vecAggrDesc) {
+    super(vecAggrDesc);
+  }
+
+  @Override
+  public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+    return new Aggregation();
+  }
+
+  @Override
+  public void aggregateInput(AggregationBuffer agg, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+    BytesColumnVector inputColumn = (BytesColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+    Aggregation myagg = (Aggregation) agg;
+
+    myagg.prepare();
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        myagg.estimator.addToEstimator(inputColumn.vector[0], inputColumn.start[0], inputColumn.length[0]);
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            myagg.estimator.addToEstimator(inputColumn.vector[i], inputColumn.start[i], inputColumn.length[i]);
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            myagg.estimator.addToEstimator(inputColumn.vector[i], inputColumn.start[i], inputColumn.length[i]);
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          myagg.estimator.addToEstimator(inputColumn.vector[0], inputColumn.start[0], inputColumn.length[0]);
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int j = 0; j < batchSize; ++j) {
+            int i = batch.selected[j];
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.vector[i], inputColumn.start[i], inputColumn.length[i]);
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.vector[i], inputColumn.start[i], inputColumn.length[i]);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private Aggregation getAggregation(VectorAggregationBufferRow[] sets, int rowid, int bufferIndex) {
+    VectorAggregationBufferRow bufferRow = sets[rowid];
+    Aggregation myagg = (Aggregation) bufferRow.getAggregationBuffer(bufferIndex);
+    myagg.prepare();
+    return myagg;
+  }
+
+  @Override
+  public void aggregateInputSelection(VectorAggregationBufferRow[] aggregationBufferSets, int aggregateIndex, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+    BytesColumnVector inputColumn = (BytesColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        for (int i = 0; i < batchSize; i++) {
+          Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+          myagg.estimator.addToEstimator(inputColumn.vector[0], inputColumn.start[0], inputColumn.length[0]);
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            Aggregation myagg = getAggregation(aggregationBufferSets, s, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[i], inputColumn.start[i], inputColumn.length[i]);
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[i], inputColumn.start[i], inputColumn.length[i]);
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          for (int i = 0; i < batchSize; i++) {
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.vector[0], inputColumn.start[0], inputColumn.length[0]);
+          }
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            if (!inputColumn.isNull[i]) {
+              Aggregation myagg = getAggregation(aggregationBufferSets, s, aggregateIndex);
+              myagg.estimator.addToEstimator(inputColumn.vector[i], inputColumn.start[i], inputColumn.length[i]);
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+              myagg.estimator.addToEstimator(inputColumn.vector[i], inputColumn.start[i], inputColumn.length[i]);
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void reset(AggregationBuffer agg) throws HiveException {
+    agg.reset();
+  }
+
+  @Override
+  public long getAggregationBufferFixedSize() {
+    return Aggregation.FIXED_SIZE;
+  }
+
+  @Override
+  public boolean matches(String name, ColumnVector.Type inputColVectorType, ColumnVector.Type outputColVectorType, GenericUDAFEvaluator.Mode mode) {
+    return name.equals("compute_bit_vector_hll") &&
+            inputColVectorType == ColumnVector.Type.BYTES &&
+            outputColVectorType == ColumnVector.Type.BYTES &&
+            (mode == GenericUDAFEvaluator.Mode.PARTIAL1 || mode == GenericUDAFEvaluator.Mode.COMPLETE);
+  }
+
+  @Override
+  public void assignRowColumn(VectorizedRowBatch batch, int batchIndex, int columnNum, AggregationBuffer agg) throws HiveException {
+    Aggregation myagg = (Aggregation) agg;
+    BytesColumnVector outputCol = (BytesColumnVector) batch.cols[columnNum];
+    if (myagg.estimator == null) {
+      outputCol.isNull[batchIndex] = true;
+      outputCol.noNulls = false;
+    } else {
+      outputCol.isNull[batchIndex] = false;
+      outputCol.isRepeating = false;
+      byte[] outputbuf = myagg.estimator.serialize();
+      outputCol.setRef(batchIndex, outputbuf, 0, outputbuf.length);
+    }
+  }
+
+  static class Aggregation implements AggregationBuffer {
+
+    private static final long FIXED_SIZE = HyperLogLog.builder().setSizeOptimized().build().lengthFor(null);
+
+    HyperLogLog estimator;
+
+    public Aggregation() {
+      reset();
+    }
+
+    @Override
+    public int getVariableSize() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void reset() {
+      estimator = null;
+    }
+
+    public void prepare() {
+      if (estimator == null) {
+        estimator = HyperLogLog.builder().setSizeOptimized().build();
+      }
+    }
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorTimestamp.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorTimestamp.java
new file mode 100644
index 0000000..227d831d
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/expressions/aggregates/VectorUDAFComputeBitVectorTimestamp.java
@@ -0,0 +1,234 @@
+/*
+ * 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.exec.vector.expressions.aggregates;
+
+import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.vector.BytesColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.ColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.TimestampColumnVector;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationBufferRow;
+import org.apache.hadoop.hive.ql.exec.vector.VectorAggregationDesc;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedRowBatch;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator;
+
+/**
+ * Generated from template VectorUDAFComputeBitVector.txt.
+ */
+
+@Description(name = "compute_bit_vector_hll",
+    value = "_FUNC_(x) - Computes bit vector for NDV computation")
+public class VectorUDAFComputeBitVectorTimestamp extends VectorAggregateExpression {
+
+  public VectorUDAFComputeBitVectorTimestamp() {
+    super();
+  }
+
+  public VectorUDAFComputeBitVectorTimestamp(VectorAggregationDesc vecAggrDesc) {
+    super(vecAggrDesc);
+  }
+
+  @Override
+  public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+    return new Aggregation();
+  }
+
+  @Override
+  public void aggregateInput(AggregationBuffer agg, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+    TimestampColumnVector inputColumn = (TimestampColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+    Aggregation myagg = (Aggregation) agg;
+
+    myagg.prepare();
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(0));
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(i));
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(i));
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(0));
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int j = 0; j < batchSize; ++j) {
+            int i = batch.selected[j];
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(i));
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(i));
+            }
+          }
+        }
+      }
+    }
+  }
+
+  private Aggregation getAggregation(VectorAggregationBufferRow[] sets, int rowid, int bufferIndex) {
+    VectorAggregationBufferRow bufferRow = sets[rowid];
+    Aggregation myagg = (Aggregation) bufferRow.getAggregationBuffer(bufferIndex);
+    myagg.prepare();
+    return myagg;
+  }
+
+  @Override
+  public void aggregateInputSelection(VectorAggregationBufferRow[] aggregationBufferSets, int aggregateIndex, VectorizedRowBatch batch) throws HiveException {
+    inputExpression.evaluate(batch);
+
+    TimestampColumnVector inputColumn = (TimestampColumnVector) batch.cols[this.inputExpression.getOutputColumnNum()];
+
+    int batchSize = batch.size;
+
+    if (batchSize == 0) {
+      return;
+    }
+
+    if (inputColumn.noNulls) {
+      if (inputColumn.isRepeating) {
+        for (int i = 0; i < batchSize; i++) {
+          Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+          myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(0));
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            Aggregation myagg = getAggregation(aggregationBufferSets, s, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(i));
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(i));
+          }
+        }
+      }
+    } else {
+      if (inputColumn.isRepeating) {
+        if (!inputColumn.isNull[0]) {
+          for (int i = 0; i < batchSize; i++) {
+            Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+            myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(0));
+          }
+        }
+      } else {
+        if (batch.selectedInUse) {
+          for (int s = 0; s < batchSize; s++) {
+            int i = batch.selected[s];
+            if (!inputColumn.isNull[i]) {
+              Aggregation myagg = getAggregation(aggregationBufferSets, s, aggregateIndex);
+              myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(i));
+            }
+          }
+        } else {
+          for (int i = 0; i < batchSize; i++) {
+            if (!inputColumn.isNull[i]) {
+              Aggregation myagg = getAggregation(aggregationBufferSets, i, aggregateIndex);
+              myagg.estimator.addToEstimator(inputColumn.getTimestampAsLong(i));
+            }
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public void reset(AggregationBuffer agg) throws HiveException {
+    agg.reset();
+  }
+
+  @Override
+  public long getAggregationBufferFixedSize() {
+    return Aggregation.FIXED_SIZE;
+  }
+
+  @Override
+  public boolean matches(String name, ColumnVector.Type inputColVectorType, ColumnVector.Type outputColVectorType, GenericUDAFEvaluator.Mode mode) {
+    return name.equals("compute_bit_vector_hll") &&
+        outputColVectorType == ColumnVector.Type.BYTES &&
+        inputColVectorType == ColumnVector.Type.TIMESTAMP &&
+        (mode == GenericUDAFEvaluator.Mode.PARTIAL1 || mode == GenericUDAFEvaluator.Mode.COMPLETE);
+  }
+
+  @Override
+  public void assignRowColumn(VectorizedRowBatch batch, int batchIndex, int columnNum, AggregationBuffer agg) throws HiveException {
+    Aggregation myagg = (Aggregation) agg;
+    BytesColumnVector outputCol = (BytesColumnVector) batch.cols[columnNum];
+    if (myagg.estimator == null) {
+      outputCol.isNull[batchIndex] = true;
+      outputCol.noNulls = false;
+    } else {
+      outputCol.isNull[batchIndex] = false;
+      outputCol.isRepeating = false;
+      byte[] outputbuf = myagg.estimator.serialize();
+      outputCol.setRef(batchIndex, outputbuf, 0, outputbuf.length);
+    }
+  }
+
+  static class Aggregation implements AggregationBuffer {
+
+    private static final long FIXED_SIZE = HyperLogLog.builder().setSizeOptimized().build().lengthFor(null);
+
+    HyperLogLog estimator;
+
+    public Aggregation() {
+      reset();
+    }
+
+    @Override
+    public int getVariableSize() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void reset() {
+      estimator = null;
+    }
+
+    public void prepare() {
+      if (estimator == null) {
+        estimator = HyperLogLog.builder().setSizeOptimized().build();
+      }
+    }
+  }
+}
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 f801b5f..165c06e 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
@@ -524,6 +524,7 @@ public class Vectorizer implements PhysicalPlanResolver {
     supportedAggregationUdfs.add("stddev_pop");
     supportedAggregationUdfs.add("stddev_samp");
     supportedAggregationUdfs.add("bloom_filter");
+    supportedAggregationUdfs.add("compute_bit_vector_hll");
   }
 
   private class VectorTaskColumnInfo {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
index d24b786..6ee3899 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/ColumnStatsSemanticAnalyzer.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.QueryState;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentException;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -457,22 +458,26 @@ public class ColumnStatsSemanticAnalyzer extends SemanticAnalyzer {
   private static void appendBitVector(StringBuilder rewrittenQueryBuilder, HiveConf conf,
       String columnName) throws SemanticException {
     String func = HiveConf.getVar(conf, HiveConf.ConfVars.HIVE_STATS_NDV_ALGO).toLowerCase();
-    rewrittenQueryBuilder.append("compute_bit_vector(")
-        .append(columnName)
-        .append(", '")
-        .append(func)
-        .append("'");
-    if ("fm".equals(func)) {
+    if ("hll".equals(func)) {
+      rewrittenQueryBuilder
+          .append("compute_bit_vector_hll(")
+          .append(columnName)
+          .append(")");
+    } else if ("fm".equals(func)) {
       int numBitVectors;
       try {
         numBitVectors = HiveStatsUtils.getNumBitVectorsForNDVEstimation(conf);
       } catch (Exception e) {
         throw new SemanticException(e.getMessage());
       }
-      rewrittenQueryBuilder.append(", ")
-          .append(numBitVectors);
+      rewrittenQueryBuilder.append("compute_bit_vector_fm(")
+          .append(columnName)
+          .append(", ")
+          .append(numBitVectors)
+          .append(")");
+    } else {
+      throw new UDFArgumentException("available ndv computation options are hll and fm. Got: " + func);
     }
-    rewrittenQueryBuilder.append(")");
   }
 
   private static void appendCountTrues(StringBuilder rewrittenQueryBuilder, HiveConf conf,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVector.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVector.java
deleted file mode 100644
index 7e91fc7..0000000
--- a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVector.java
+++ /dev/null
@@ -1,559 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hive.ql.udf.generic;
-
-import org.apache.hadoop.hive.common.classification.InterfaceAudience;
-import org.apache.hadoop.hive.common.ndv.NumDistinctValueEstimator;
-import org.apache.hadoop.hive.common.type.HiveDecimal;
-import org.apache.hadoop.hive.ql.exec.Description;
-import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
-import org.apache.hadoop.hive.ql.metadata.HiveException;
-import org.apache.hadoop.hive.ql.parse.SemanticException;
-import org.apache.hadoop.hive.ql.stats.ColStatsProcessor.ColumnStatsType;
-import org.apache.hadoop.hive.ql.util.JavaDataModel;
-import org.apache.hadoop.hive.serde2.io.DateWritableV2;
-import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
-import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
-import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
-import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
-import org.apache.hadoop.io.BytesWritable;
-
-import static org.apache.hadoop.hive.common.ndv.NumDistinctValueEstimatorFactory.getEmptyNumDistinctValueEstimator;
-import static org.apache.hadoop.hive.common.ndv.NumDistinctValueEstimatorFactory.getNumDistinctValueEstimator;
-
-/**
- * GenericUDAFComputeBitVector. This UDAF will compute a bit vector using the
- * algorithm provided as a parameter. The ndv_compute_bit_vector function can
- * be used on top of it to extract an estimate of the ndv from it.
- */
-@Description(name = "compute_bit_vector",
-      value = "_FUNC_(x) - Computes bit vector for NDV computation.")
-public class GenericUDAFComputeBitVector extends AbstractGenericUDAFResolver {
-
-  @Override
-  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
-      throws SemanticException {
-    if (parameters.length < 2 ) {
-      throw new UDFArgumentTypeException(parameters.length - 1,
-          "Exactly 2 (col + hll) or 3 (col + fm + #bitvectors) arguments are expected.");
-    }
-
-    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
-      throw new UDFArgumentTypeException(0,
-          "Only primitive type arguments are accepted but "
-          + parameters[0].getTypeName() + " is passed.");
-    }
-
-    ColumnStatsType cst = ColumnStatsType.getColumnStatsType(((PrimitiveTypeInfo) parameters[0]));
-    switch (cst) {
-    case LONG:
-      return new GenericUDAFLongStatsEvaluator();
-    case DOUBLE:
-      return new GenericUDAFDoubleStatsEvaluator();
-    case STRING:
-      return new GenericUDAFStringStatsEvaluator();
-    case DECIMAL:
-      return new GenericUDAFDecimalStatsEvaluator();
-    case DATE:
-      return new GenericUDAFDateStatsEvaluator();
-    case TIMESTAMP:
-      return new GenericUDAFTimestampStatsEvaluator();
-    default:
-      throw new UDFArgumentTypeException(0,
-          "Type argument " + parameters[0].getTypeName() + " not valid");
-    }
-  }
-
-  public static abstract class GenericUDAFNumericStatsEvaluator<V, OI extends PrimitiveObjectInspector>
-      extends GenericUDAFEvaluator {
-
-    protected final static int MAX_BIT_VECTORS = 1024;
-
-    /* Object Inspector corresponding to the input parameter.
-     */
-    protected transient PrimitiveObjectInspector inputOI;
-    protected transient PrimitiveObjectInspector funcOI;
-    protected transient PrimitiveObjectInspector numVectorsOI;
-
-    /* Object Inspector corresponding to the bitvector.
-     */
-    protected transient BinaryObjectInspector ndvFieldOI;
-
-    /* Partial aggregation result returned by TerminatePartial.
-     */
-    protected transient BytesWritable partialResult;
-
-    /* Output of final result of the aggregation.
-     */
-    protected transient BytesWritable result;
-
-    @Override
-    public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-      super.init(m, parameters);
-
-      // initialize input
-      if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
-        inputOI = (PrimitiveObjectInspector) parameters[0];
-        funcOI = (PrimitiveObjectInspector) parameters[1];
-        if (parameters.length > 2) {
-          numVectorsOI = (PrimitiveObjectInspector) parameters[2];
-        }
-      } else {
-        ndvFieldOI = (BinaryObjectInspector) parameters[0];
-      }
-
-      // initialize output
-      if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
-        partialResult = new BytesWritable();
-        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
-      } else {
-        result = new BytesWritable();
-        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
-      }
-    }
-
-    @Override
-    public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-      NumericStatsAgg myagg = (NumericStatsAgg) agg;
-
-      if (myagg.numDV == null) {
-        int numVectors = 0;
-        // func may be null when GBY op is closing.
-        // see mvn test -Dtest=TestMiniTezCliDriver -Dqfile=explainuser_3.q
-        // original behavior is to create FMSketch
-        String func = parameters[1] == null ? "fm" : PrimitiveObjectInspectorUtils.getString(
-            parameters[1], funcOI);
-        if (parameters.length == 3) {
-          numVectors = parameters[2] == null ? 0 : PrimitiveObjectInspectorUtils.getInt(
-              parameters[2], numVectorsOI);
-          if (numVectors > MAX_BIT_VECTORS) {
-            throw new HiveException("The maximum allowed value for number of bit vectors " + " is "
-                + MAX_BIT_VECTORS + ", but was passed " + numVectors + " bit vectors");
-          }
-        }
-        myagg.initNDVEstimator(func, numVectors);
-      }
-
-      if (parameters[0] != null) {
-        myagg.update(parameters[0], inputOI);
-      }
-    }
-
-    @Override
-    public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-      if (partial != null) {
-        NumericStatsAgg myagg = (NumericStatsAgg) agg;
-        // Merge numDistinctValue Estimators
-        byte[] buf = ndvFieldOI.getPrimitiveJavaObject(partial);
-        if (buf != null && buf.length != 0) {
-          if (myagg.numDV == null) {
-            myagg.numDV = getNumDistinctValueEstimator(buf);
-          } else {
-            myagg.numDV.mergeEstimators(getNumDistinctValueEstimator(buf));
-          }
-        }
-      }
-    }
-
-    @Override
-    public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-      return ((NumericStatsAgg) agg).serializePartial(partialResult);
-    }
-
-    @Override
-    public Object terminate(AggregationBuffer agg) throws HiveException {
-      return ((NumericStatsAgg) agg).serialize(result);
-    }
-
-    public abstract class NumericStatsAgg extends AbstractAggregationBuffer {
-
-      public NumDistinctValueEstimator numDV;    /* Distinct value estimator */
-
-      @Override
-      public int estimate() {
-        JavaDataModel model = JavaDataModel.get();
-        return (numDV == null) ?
-            lengthFor(model) : numDV.lengthFor(model);
-      }
-
-      protected void initNDVEstimator(String func, int numBitVectors) {
-        numDV = getEmptyNumDistinctValueEstimator(func, numBitVectors);
-      }
-
-      protected abstract void update(Object p, PrimitiveObjectInspector inputOI);
-
-      protected Object serialize(BytesWritable result) {
-        if (numDV != null) {
-          byte[] buf = numDV.serialize();
-          result.set(buf, 0, buf.length);
-        }
-        return result;
-      }
-
-      protected Object serializePartial(BytesWritable result) {
-        if (numDV != null) {
-          // Serialize numDistinctValue Estimator
-          byte[] buf = numDV.serialize();
-          result.set(buf, 0, buf.length);
-        }
-        return result;
-      }
-
-      public void reset() throws HiveException {
-        numDV = null;
-      }
-    };
-  }
-
-  /**
-   * GenericUDAFLongStatsEvaluator.
-   *
-   */
-  public static class GenericUDAFLongStatsEvaluator
-      extends GenericUDAFNumericStatsEvaluator<Long, LongObjectInspector> {
-
-    @AggregationType(estimable = true)
-    public class LongStatsAgg extends NumericStatsAgg {
-      @Override
-      public int estimate() {
-        JavaDataModel model = JavaDataModel.get();
-        return super.estimate() + model.primitive2() * 2;
-      }
-
-      @Override
-      protected void update(Object p, PrimitiveObjectInspector inputOI) {
-        long v = PrimitiveObjectInspectorUtils.getLong(p, inputOI);
-        numDV.addToEstimator(v);
-      }
-    };
-
-    @Override
-    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-      AggregationBuffer result = new LongStatsAgg();
-      reset(result);
-      return result;
-    }
-
-    @Override
-    public void reset(AggregationBuffer agg) throws HiveException {
-      ((NumericStatsAgg)agg).reset();
-    }
-  }
-
-  /**
-   * GenericUDAFDoubleStatsEvaluator.
-   */
-  public static class GenericUDAFDoubleStatsEvaluator
-      extends GenericUDAFNumericStatsEvaluator<Double, DoubleObjectInspector> {
-
-    @AggregationType(estimable = true)
-    public class DoubleStatsAgg extends NumericStatsAgg {
-      @Override
-      public int estimate() {
-        JavaDataModel model = JavaDataModel.get();
-        return super.estimate() + model.primitive2() * 2;
-      }
-
-      @Override
-      protected void update(Object p, PrimitiveObjectInspector inputOI) {
-        double v = PrimitiveObjectInspectorUtils.getDouble(p, inputOI);
-        numDV.addToEstimator(v);
-      }
-    };
-
-    @Override
-    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-      AggregationBuffer result = new DoubleStatsAgg();
-      reset(result);
-      return result;
-    }
-
-    @Override
-    public void reset(AggregationBuffer agg) throws HiveException {
-      ((NumericStatsAgg)agg).reset();
-    }
-  }
-
-  public static class GenericUDAFDecimalStatsEvaluator
-      extends GenericUDAFNumericStatsEvaluator<HiveDecimal, HiveDecimalObjectInspector> {
-
-    @AggregationType(estimable = true)
-    public class DecimalStatsAgg extends NumericStatsAgg {
-      @Override
-      public int estimate() {
-        JavaDataModel model = JavaDataModel.get();
-        return super.estimate() + model.lengthOfDecimal() * 2;
-      }
-
-      @Override
-      protected void update(Object p, PrimitiveObjectInspector inputOI) {
-        HiveDecimal v = PrimitiveObjectInspectorUtils.getHiveDecimal(p, inputOI);
-        numDV.addToEstimator(v);
-      }
-    };
-
-    @Override
-    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-      AggregationBuffer result = new DecimalStatsAgg();
-      reset(result);
-      return result;
-    }
-
-    @Override
-    public void reset(AggregationBuffer agg) throws HiveException {
-      ((NumericStatsAgg)agg).reset();
-    }
-  }
-
-  /**
-   * GenericUDAFDateStatsEvaluator.
-   */
-  public static class GenericUDAFDateStatsEvaluator
-      extends GenericUDAFNumericStatsEvaluator<DateWritableV2, DateObjectInspector> {
-
-    @AggregationType(estimable = true)
-    public class DateStatsAgg extends NumericStatsAgg {
-      @Override
-      public int estimate() {
-        JavaDataModel model = JavaDataModel.get();
-        return super.estimate() + model.primitive2() * 2;
-      }
-
-      @Override
-      protected void update(Object p, PrimitiveObjectInspector inputOI) {
-        // DateWritableV2 is mutable, DateStatsAgg needs its own copy
-        DateWritableV2 v = new DateWritableV2((DateWritableV2) inputOI.getPrimitiveWritableObject(p));
-        numDV.addToEstimator(v.getDays());
-      }
-    };
-
-    @Override
-    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-      AggregationBuffer result = new DateStatsAgg();
-      reset(result);
-      return result;
-    }
-
-    @Override
-    public void reset(AggregationBuffer agg) throws HiveException {
-      ((NumericStatsAgg)agg).reset();
-    }
-  }
-
-  /**
-   * GenericUDAFTimestampStatsEvaluator.
-   */
-  public static class GenericUDAFTimestampStatsEvaluator
-      extends GenericUDAFNumericStatsEvaluator<TimestampWritableV2, TimestampObjectInspector> {
-
-    @AggregationType(estimable = true)
-    public class TimestampStatsAgg extends NumericStatsAgg {
-      @Override
-      public int estimate() {
-        JavaDataModel model = JavaDataModel.get();
-        return super.estimate() + model.primitive2() * 2;
-      }
-
-      @Override
-      protected void update(Object p, PrimitiveObjectInspector inputOI) {
-        // TimestampWritableV2 is mutable, TimestampStatsAgg needs its own copy
-        TimestampWritableV2 v = new TimestampWritableV2((TimestampWritableV2) inputOI.getPrimitiveWritableObject(p));
-        numDV.addToEstimator(v.getSeconds());
-      }
-    };
-
-    @Override
-    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-      AggregationBuffer result = new TimestampStatsAgg();
-      reset(result);
-      return result;
-    }
-
-    @Override
-    public void reset(AggregationBuffer agg) throws HiveException {
-      ((NumericStatsAgg)agg).reset();
-    }
-  }
-
-  /**
-   * GenericUDAFStringStatsEvaluator.
-   */
-  public static class GenericUDAFStringStatsEvaluator extends GenericUDAFEvaluator {
-
-    private final static int MAX_BIT_VECTORS = 1024;
-
-    /* Object Inspector corresponding to the input parameter.
-     */
-    private transient PrimitiveObjectInspector inputOI;
-    private transient PrimitiveObjectInspector funcOI;
-    private transient PrimitiveObjectInspector numVectorsOI;
-
-    /* Object Inspector corresponding to the bitvector
-     */
-    private transient BinaryObjectInspector ndvFieldOI;
-
-    /* Partial aggregation result returned by TerminatePartial.
-     */
-    private transient BytesWritable partialResult;
-
-    /* Output of final result of the aggregation
-     */
-    private transient BytesWritable result;
-
-    @Override
-    public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
-      super.init(m, parameters);
-
-      // initialize input
-      if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
-        inputOI = (PrimitiveObjectInspector) parameters[0];
-        funcOI = (PrimitiveObjectInspector) parameters[1];
-        if (parameters.length > 2) {
-          numVectorsOI = (PrimitiveObjectInspector) parameters[2];
-        }
-      } else {
-        ndvFieldOI = (BinaryObjectInspector) parameters[0];
-      }
-
-      // initialize output
-      if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
-        partialResult = new BytesWritable();
-        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
-      } else {
-        result = new BytesWritable();
-        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
-      }
-    }
-
-    @AggregationType(estimable = true)
-    public static class StringStatsAgg extends AbstractAggregationBuffer {
-      public NumDistinctValueEstimator numDV;      /* Distinct value estimator */
-      public boolean firstItem;
-      @Override
-      public int estimate() {
-        JavaDataModel model = JavaDataModel.get();
-        return (numDV == null) ?
-            lengthFor(model) : numDV.lengthFor(model);      }
-    };
-
-    @Override
-    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
-      StringStatsAgg result = new StringStatsAgg();
-      reset(result);
-      return result;
-    }
-
-    public void initNDVEstimator(StringStatsAgg aggBuffer, String func, int numBitVectors) {
-      aggBuffer.numDV = getEmptyNumDistinctValueEstimator(func, numBitVectors);
-      aggBuffer.numDV.reset();
-    }
-
-    @Override
-    public void reset(AggregationBuffer agg) throws HiveException {
-      StringStatsAgg myagg = (StringStatsAgg) agg;
-      myagg.firstItem = true;
-    }
-
-    @Override
-    public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
-      Object p = parameters[0];
-      StringStatsAgg myagg = (StringStatsAgg) agg;
-
-      if (myagg.firstItem) {
-        int numVectors = 0;
-        String func = parameters[1] == null ? "fm" : PrimitiveObjectInspectorUtils.getString(
-            parameters[1], funcOI);
-        if (parameters.length > 2) {
-          numVectors = PrimitiveObjectInspectorUtils.getInt(parameters[2], numVectorsOI);
-          if (numVectors > MAX_BIT_VECTORS) {
-            throw new HiveException("The maximum allowed value for number of bit vectors " + " is "
-                + MAX_BIT_VECTORS + " , but was passed " + numVectors + " bit vectors");
-          }
-        }
-
-        initNDVEstimator(myagg, func, numVectors);
-        myagg.firstItem = false;
-      }
-
-      String v = PrimitiveObjectInspectorUtils.getString(p, inputOI);
-      if (v != null) {
-        // Add string value to NumDistinctValue Estimator
-        myagg.numDV.addToEstimator(v);
-      }
-    }
-
-    @Override
-    public void merge(AggregationBuffer agg, Object partial) throws HiveException {
-      if (partial != null) {
-        StringStatsAgg myagg = (StringStatsAgg) agg;
-
-        // Merge numDistinctValue Estimators
-        byte[] buf = ndvFieldOI.getPrimitiveJavaObject(partial);
-
-        if (buf != null && buf.length != 0) {
-          if (myagg.numDV == null) {
-            myagg.numDV = getNumDistinctValueEstimator(buf);
-          } else {
-            myagg.numDV.mergeEstimators(getNumDistinctValueEstimator(buf));
-          }
-        }
-      }
-    }
-
-    @Override
-    public Object terminatePartial(AggregationBuffer agg) throws HiveException {
-      StringStatsAgg myagg = (StringStatsAgg) agg;
-      // Serialize numDistinctValue Estimator
-      if (myagg.numDV != null) {
-        byte[] buf = myagg.numDV.serialize();
-        partialResult.set(buf, 0, buf.length);
-      }
-      return partialResult;
-    }
-
-    @Override
-    public Object terminate(AggregationBuffer agg) throws HiveException {
-      StringStatsAgg myagg = (StringStatsAgg) agg;
-      if (myagg.numDV != null) {
-        byte[] buf = myagg.numDV.serialize();
-        result.set(buf, 0, buf.length);
-      }
-      return result;
-    }
-  }
-
-  @InterfaceAudience.LimitedPrivate(value = { "Hive" })
-  static int lengthFor(JavaDataModel model) {
-    int length = model.object();
-    // HiveConf hive.stats.ndv.error default produces 16
-    length += model.array() * 3; // three array
-    length += model.primitive1() * 16 * 2; // two int array
-    length += (model.object() + model.array() + model.primitive1() + model.primitive2())
-        * 16; // bitset array
-    return length;
-  }
-}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorBase.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorBase.java
new file mode 100644
index 0000000..782fff1
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorBase.java
@@ -0,0 +1,291 @@
+/*
+ * 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.udf.generic;
+
+import org.apache.hadoop.hive.common.classification.InterfaceAudience;
+import org.apache.hadoop.hive.common.ndv.NumDistinctValueEstimator;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.util.JavaDataModel;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.io.BytesWritable;
+
+import static org.apache.hadoop.hive.common.ndv.NumDistinctValueEstimatorFactory.getNumDistinctValueEstimator;
+
+public abstract class GenericUDAFComputeBitVectorBase extends AbstractGenericUDAFResolver {
+
+  public static abstract class NumericStatsEvaluatorBase<V, OI extends PrimitiveObjectInspector>
+      extends GenericUDAFEvaluator {
+
+    /* Object Inspector corresponding to the input parameter.
+     */
+    protected transient PrimitiveObjectInspector inputOI;
+
+    /* Object Inspector corresponding to the bitvector.
+     */
+    protected transient BinaryObjectInspector ndvFieldOI;
+
+    /* Partial aggregation result returned by TerminatePartial.
+     */
+    protected transient BytesWritable partialResult;
+
+    /* Output of final result of the aggregation.
+     */
+    protected transient BytesWritable result;
+
+    @Override
+    public void reset(AggregationBuffer agg) throws HiveException {
+      ((NumericStatsAgg)agg).reset();
+    }
+
+    @Override
+    public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+      if (partial != null) {
+        NumericStatsAgg myagg = (NumericStatsAgg) agg;
+        // Merge numDistinctValue Estimators
+        byte[] buf = ndvFieldOI.getPrimitiveJavaObject(partial);
+        if (buf != null && buf.length != 0) {
+          if (myagg.numDV == null) {
+            myagg.numDV = getNumDistinctValueEstimator(buf);
+          } else {
+            myagg.numDV.mergeEstimators(getNumDistinctValueEstimator(buf));
+          }
+        }
+      }
+    }
+
+    @Override
+    public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+      return ((NumericStatsAgg) agg).serializePartial(partialResult);
+    }
+
+    @Override
+    public Object terminate(AggregationBuffer agg) throws HiveException {
+      return ((NumericStatsAgg) agg).serialize(result);
+    }
+  }
+
+  public static abstract class StringStatsEvaluatorBase extends GenericUDAFEvaluator {
+
+    /* Object Inspector corresponding to the input parameter.
+     */
+    protected transient PrimitiveObjectInspector inputOI;
+
+    /* Object Inspector corresponding to the bitvector
+     */
+    protected transient BinaryObjectInspector ndvFieldOI;
+
+    /* Partial aggregation result returned by TerminatePartial.
+     */
+    protected transient BytesWritable partialResult;
+
+    /* Output of final result of the aggregation
+     */
+    protected transient BytesWritable result;
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      StringStatsAgg result = new StringStatsAgg();
+      reset(result);
+      return result;
+    }
+
+    @Override
+    public void reset(AggregationBuffer agg) throws HiveException {
+      StringStatsAgg myagg = (StringStatsAgg) agg;
+      myagg.firstItem = true;
+    }
+
+    @Override
+    public void merge(AggregationBuffer agg, Object partial) throws HiveException {
+      if (partial != null) {
+        StringStatsAgg myagg = (StringStatsAgg) agg;
+
+        // Merge numDistinctValue Estimators
+        byte[] buf = ndvFieldOI.getPrimitiveJavaObject(partial);
+
+        if (buf != null && buf.length != 0) {
+          if (myagg.numDV == null) {
+            myagg.numDV = getNumDistinctValueEstimator(buf);
+          } else {
+            myagg.numDV.mergeEstimators(getNumDistinctValueEstimator(buf));
+          }
+        }
+      }
+    }
+
+    @Override
+    public Object terminatePartial(AggregationBuffer agg) throws HiveException {
+      StringStatsAgg myagg = (StringStatsAgg) agg;
+      // Serialize numDistinctValue Estimator
+      if (myagg.numDV != null) {
+        byte[] buf = myagg.numDV.serialize();
+        partialResult.set(buf, 0, buf.length);
+      }
+      return partialResult;
+    }
+
+    @Override
+    public Object terminate(AggregationBuffer agg) throws HiveException {
+      StringStatsAgg myagg = (StringStatsAgg) agg;
+      if (myagg.numDV != null) {
+        byte[] buf = myagg.numDV.serialize();
+        result.set(buf, 0, buf.length);
+      }
+      return result;
+    }
+  }
+
+  public static abstract class NumericStatsAgg extends GenericUDAFEvaluator.AbstractAggregationBuffer {
+
+    public NumDistinctValueEstimator numDV;    /* Distinct value estimator */
+
+    @Override
+    public int estimate() {
+      JavaDataModel model = JavaDataModel.get();
+      return (numDV == null) ?
+          lengthFor(model) : numDV.lengthFor(model);
+    }
+
+    protected abstract void update(Object p, PrimitiveObjectInspector inputOI);
+
+    protected Object serialize(BytesWritable result) {
+      if (numDV != null) {
+        byte[] buf = numDV.serialize();
+        result.set(buf, 0, buf.length);
+      }
+      return result;
+    }
+
+    protected Object serializePartial(BytesWritable result) {
+      if (numDV != null) {
+        // Serialize numDistinctValue Estimator
+        byte[] buf = numDV.serialize();
+        result.set(buf, 0, buf.length);
+      }
+      return result;
+    }
+
+    public void reset() throws HiveException {
+      numDV = null;
+    }
+  };
+
+  @GenericUDAFEvaluator.AggregationType(estimable = true)
+  public static class LongStatsAgg extends NumericStatsAgg {
+    @Override
+    public int estimate() {
+      JavaDataModel model = JavaDataModel.get();
+      return super.estimate() + model.primitive2() * 2;
+    }
+
+    @Override
+    protected void update(Object p, PrimitiveObjectInspector inputOI) {
+      long v = PrimitiveObjectInspectorUtils.getLong(p, inputOI);
+      numDV.addToEstimator(v);
+    }
+  };
+
+  @GenericUDAFEvaluator.AggregationType(estimable = true)
+  public static class DoubleStatsAgg extends NumericStatsAgg {
+    @Override
+    public int estimate() {
+      JavaDataModel model = JavaDataModel.get();
+      return super.estimate() + model.primitive2() * 2;
+    }
+
+    @Override
+    protected void update(Object p, PrimitiveObjectInspector inputOI) {
+      double v = PrimitiveObjectInspectorUtils.getDouble(p, inputOI);
+      numDV.addToEstimator(v);
+    }
+  };
+
+  @GenericUDAFEvaluator.AggregationType(estimable = true)
+  public static class DecimalStatsAgg extends NumericStatsAgg {
+    @Override
+    public int estimate() {
+      JavaDataModel model = JavaDataModel.get();
+      return super.estimate() + model.lengthOfDecimal() * 2;
+    }
+
+    @Override
+    protected void update(Object p, PrimitiveObjectInspector inputOI) {
+      HiveDecimal v = PrimitiveObjectInspectorUtils.getHiveDecimal(p, inputOI);
+      numDV.addToEstimator(v);
+    }
+  };
+
+  @GenericUDAFEvaluator.AggregationType(estimable = true)
+  public static class DateStatsAgg extends NumericStatsAgg {
+    @Override
+    public int estimate() {
+      JavaDataModel model = JavaDataModel.get();
+      return super.estimate() + model.primitive2() * 2;
+    }
+
+    @Override
+    protected void update(Object p, PrimitiveObjectInspector inputOI) {
+      // DateWritableV2 is mutable, DateStatsAgg needs its own copy
+      DateWritableV2 v = new DateWritableV2((DateWritableV2) inputOI.getPrimitiveWritableObject(p));
+      numDV.addToEstimator(v.getDays());
+    }
+  };
+
+  @GenericUDAFEvaluator.AggregationType(estimable = true)
+  public static class TimestampStatsAgg extends NumericStatsAgg {
+    @Override
+    public int estimate() {
+      JavaDataModel model = JavaDataModel.get();
+      return super.estimate() + model.primitive2() * 2;
+    }
+
+    @Override
+    protected void update(Object p, PrimitiveObjectInspector inputOI) {
+      // TimestampWritableV2 is mutable, TimestampStatsAgg needs its own copy
+      TimestampWritableV2 v = new TimestampWritableV2((TimestampWritableV2) inputOI.getPrimitiveWritableObject(p));
+      numDV.addToEstimator(v.getSeconds());
+    }
+  };
+
+  @GenericUDAFEvaluator.AggregationType(estimable = true)
+  public static class StringStatsAgg extends GenericUDAFEvaluator.AbstractAggregationBuffer {
+    public NumDistinctValueEstimator numDV;      /* Distinct value estimator */
+    public boolean firstItem;
+    @Override
+    public int estimate() {
+      JavaDataModel model = JavaDataModel.get();
+      return (numDV == null) ?
+          lengthFor(model) : numDV.lengthFor(model);      }
+  };
+
+  @InterfaceAudience.LimitedPrivate(value = { "Hive" })
+  static int lengthFor(JavaDataModel model) {
+    int length = model.object();
+    // HiveConf hive.stats.ndv.error default produces 16
+    length += model.array() * 3; // three array
+    length += model.primitive1() * 16 * 2; // two int array
+    length += (model.object() + model.array() + model.primitive1() + model.primitive2())
+        * 16; // bitset array
+    return length;
+  }
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorFMSketch.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorFMSketch.java
new file mode 100644
index 0000000..b346795
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorFMSketch.java
@@ -0,0 +1,253 @@
+/*
+ * 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.udf.generic;
+
+import org.apache.hadoop.hive.common.ndv.fm.FMSketch;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.stats.ColStatsProcessor.ColumnStatsType;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.BytesWritable;
+
+/**
+ * GenericUDAFComputeBitVector. This UDAF will compute a bit vector using
+ * FMSketch algorithm. The ndv_compute_bit_vector function can
+ * be used on top of it to extract an estimate of the ndv from it.
+ */
+@Description(name = "compute_bit_vector_fm",
+    value = "_FUNC_(x) - Computes bit vector for NDV computation.")
+public class GenericUDAFComputeBitVectorFMSketch extends GenericUDAFComputeBitVectorBase {
+
+  @Override
+  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+      throws SemanticException {
+    if (parameters.length != 2 ) {
+      throw new UDFArgumentTypeException(parameters.length - 1,
+          "Exactly 2 (col + #bitvectors) arguments are expected.");
+    }
+
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+      throw new UDFArgumentTypeException(0,
+          "Only primitive type arguments are accepted but "
+              + parameters[0].getTypeName() + " is passed.");
+    }
+
+    ColumnStatsType cst = ColumnStatsType.getColumnStatsType(((PrimitiveTypeInfo) parameters[0]));
+    switch (cst) {
+      case LONG:
+        return new GenericUDAFLongStatsEvaluator();
+      case DOUBLE:
+        return new GenericUDAFDoubleStatsEvaluator();
+      case STRING:
+        return new GenericUDAFStringStatsEvaluator();
+      case DECIMAL:
+        return new GenericUDAFDecimalStatsEvaluator();
+      case DATE:
+        return new GenericUDAFDateStatsEvaluator();
+      case TIMESTAMP:
+        return new GenericUDAFTimestampStatsEvaluator();
+      default:
+        throw new UDFArgumentTypeException(0,
+            "Type argument " + parameters[0].getTypeName() + " not valid");
+    }
+  }
+
+  public static abstract class NumericStatsEvaluator<V, OI extends PrimitiveObjectInspector>
+      extends NumericStatsEvaluatorBase {
+
+    protected final static int MAX_BIT_VECTORS = 1024;
+
+    protected transient PrimitiveObjectInspector numVectorsOI;
+
+    @Override
+    public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+      super.init(m, parameters);
+
+      // initialize input
+      if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+        inputOI = (PrimitiveObjectInspector) parameters[0];
+        numVectorsOI = (PrimitiveObjectInspector) parameters[1];
+      } else {
+        ndvFieldOI = (BinaryObjectInspector) parameters[0];
+      }
+
+      // initialize output
+      if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+        partialResult = new BytesWritable();
+        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+      } else {
+        result = new BytesWritable();
+        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+      }
+    }
+
+    @Override
+    public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+      NumericStatsAgg myagg = (NumericStatsAgg) agg;
+
+      if (myagg.numDV == null) {
+        int numVectors = 0;
+        if (parameters.length == 2) {
+          numVectors = parameters[1] == null ? 0 : PrimitiveObjectInspectorUtils.getInt(
+              parameters[1], numVectorsOI);
+          if (numVectors > MAX_BIT_VECTORS) {
+            throw new HiveException("The maximum allowed value for number of bit vectors " + " is "
+                + MAX_BIT_VECTORS + ", but was passed " + numVectors + " bit vectors");
+          }
+        }
+        myagg.numDV = new FMSketch(numVectors);
+      }
+
+      if (parameters[0] != null) {
+        myagg.update(parameters[0], inputOI);
+      }
+    }
+  }
+
+  public static class GenericUDAFStringStatsEvaluator extends StringStatsEvaluatorBase {
+
+    private final static int MAX_BIT_VECTORS = 1024;
+
+    private transient PrimitiveObjectInspector funcOI;
+    private transient PrimitiveObjectInspector numVectorsOI;
+
+    @Override
+    public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+      super.init(m, parameters);
+
+      // initialize input
+      if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+        inputOI = (PrimitiveObjectInspector) parameters[0];
+        funcOI = (PrimitiveObjectInspector) parameters[1];
+        if (parameters.length > 2) {
+          numVectorsOI = (PrimitiveObjectInspector) parameters[2];
+        }
+      } else {
+        ndvFieldOI = (BinaryObjectInspector) parameters[0];
+      }
+
+      // initialize output
+      if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+        partialResult = new BytesWritable();
+        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+      } else {
+        result = new BytesWritable();
+        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+      }
+    }
+
+    @Override
+    public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+      Object p = parameters[0];
+      StringStatsAgg myagg = (StringStatsAgg) agg;
+
+      if (myagg.firstItem) {
+        int numVectors = 0;
+        if (parameters.length > 1) {
+          numVectors = PrimitiveObjectInspectorUtils.getInt(parameters[1], numVectorsOI);
+          if (numVectors > MAX_BIT_VECTORS) {
+            throw new HiveException("The maximum allowed value for number of bit vectors " + " is "
+                + MAX_BIT_VECTORS + " , but was passed " + numVectors + " bit vectors");
+          }
+        }
+
+        myagg.numDV = new FMSketch(numVectors);
+        myagg.numDV.reset();
+        myagg.firstItem = false;
+      }
+
+      String v = PrimitiveObjectInspectorUtils.getString(p, inputOI);
+      if (v != null) {
+        // Add string value to NumDistinctValue Estimator
+        myagg.numDV.addToEstimator(v);
+      }
+    }
+  }
+
+  public static class GenericUDAFLongStatsEvaluator
+      extends NumericStatsEvaluator<Long, LongObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.LongStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  public static class GenericUDAFDoubleStatsEvaluator
+      extends NumericStatsEvaluator<Double, DoubleObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.DoubleStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  public static class GenericUDAFDecimalStatsEvaluator
+      extends NumericStatsEvaluator<HiveDecimal, HiveDecimalObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.DecimalStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  public static class GenericUDAFDateStatsEvaluator
+      extends NumericStatsEvaluator<DateWritableV2, DateObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.DateStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  public static class GenericUDAFTimestampStatsEvaluator
+      extends NumericStatsEvaluator<TimestampWritableV2, TimestampObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.TimestampStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+}
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorHLL.java b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorHLL.java
new file mode 100644
index 0000000..4ff5a90
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFComputeBitVectorHLL.java
@@ -0,0 +1,251 @@
+/*
+ * 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.udf.generic;
+
+import org.apache.hadoop.hive.common.ndv.hll.HyperLogLog;
+import org.apache.hadoop.hive.common.type.HiveDecimal;
+import org.apache.hadoop.hive.ql.exec.Description;
+import org.apache.hadoop.hive.ql.exec.UDFArgumentTypeException;
+import org.apache.hadoop.hive.ql.exec.vector.VectorizedUDAFs;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFComputeBitVectorDecimal;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFComputeBitVectorDouble;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFComputeBitVectorFinal;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFComputeBitVectorLong;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFComputeBitVectorString;
+import org.apache.hadoop.hive.ql.exec.vector.expressions.aggregates.VectorUDAFComputeBitVectorTimestamp;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.parse.SemanticException;
+import org.apache.hadoop.hive.ql.stats.ColStatsProcessor.ColumnStatsType;
+import org.apache.hadoop.hive.serde2.io.DateWritableV2;
+import org.apache.hadoop.hive.serde2.io.TimestampWritableV2;
+import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.BinaryObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.DoubleObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.LongObjectInspector;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorFactory;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.PrimitiveObjectInspectorUtils;
+import org.apache.hadoop.hive.serde2.objectinspector.primitive.TimestampObjectInspector;
+import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo;
+import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo;
+import org.apache.hadoop.io.BytesWritable;
+
+/**
+ * GenericUDAFComputeBitVector. This UDAF will compute a bit vector using
+ * HyperLogLog algorithm. The ndv_compute_bit_vector function can
+ * be used on top of it to extract an estimate of the ndv from it.
+ */
+@Description(name = "compute_bit_vector_hll",
+    value = "_FUNC_(x) - Computes bit vector for NDV computation.")
+public class GenericUDAFComputeBitVectorHLL extends GenericUDAFComputeBitVectorBase {
+
+  @Override
+  public GenericUDAFEvaluator getEvaluator(TypeInfo[] parameters)
+      throws SemanticException {
+    if (parameters.length != 1 ) {
+      throw new UDFArgumentTypeException(parameters.length - 1,
+          "Exactly 1 (col) argument is expected.");
+    }
+
+    if (parameters[0].getCategory() != ObjectInspector.Category.PRIMITIVE) {
+      throw new UDFArgumentTypeException(0,
+          "Only primitive type arguments are accepted but "
+              + parameters[0].getTypeName() + " is passed.");
+    }
+
+    ColumnStatsType cst = ColumnStatsType.getColumnStatsType(((PrimitiveTypeInfo) parameters[0]));
+    switch (cst) {
+      case LONG:
+        return new GenericUDAFLongStatsEvaluator();
+      case DOUBLE:
+        return new GenericUDAFDoubleStatsEvaluator();
+      case STRING:
+        return new GenericUDAFStringStatsEvaluator();
+      case DECIMAL:
+        return new GenericUDAFDecimalStatsEvaluator();
+      case DATE:
+        return new GenericUDAFDateStatsEvaluator();
+      case TIMESTAMP:
+        return new GenericUDAFTimestampStatsEvaluator();
+      default:
+        throw new UDFArgumentTypeException(0,
+            "Type argument " + parameters[0].getTypeName() + " not valid");
+    }
+  }
+
+  public static abstract class GenericUDAFNumericStatsEvaluator<V, OI extends PrimitiveObjectInspector>
+      extends NumericStatsEvaluatorBase {
+
+    @Override
+    public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+      super.init(m, parameters);
+
+      // initialize input
+      if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+        inputOI = (PrimitiveObjectInspector) parameters[0];
+      } else {
+        ndvFieldOI = (BinaryObjectInspector) parameters[0];
+      }
+
+      // initialize output
+      if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+        partialResult = new BytesWritable();
+        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+      } else {
+        result = new BytesWritable();
+        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+      }
+    }
+
+    @Override
+    public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+      GenericUDAFComputeBitVectorBase.NumericStatsAgg myagg = (GenericUDAFComputeBitVectorBase.NumericStatsAgg) agg;
+
+      if (myagg.numDV == null) {
+        myagg.numDV = HyperLogLog.builder().setSizeOptimized().build();
+      }
+
+      if (parameters[0] != null) {
+        myagg.update(parameters[0], inputOI);
+      }
+    }
+  }
+
+  @VectorizedUDAFs({
+      VectorUDAFComputeBitVectorLong.class,
+      VectorUDAFComputeBitVectorFinal.class
+  })
+  public static class GenericUDAFLongStatsEvaluator
+      extends GenericUDAFNumericStatsEvaluator<Long, LongObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.LongStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  @VectorizedUDAFs({
+      VectorUDAFComputeBitVectorDouble.class,
+      VectorUDAFComputeBitVectorFinal.class
+  })
+  public static class GenericUDAFDoubleStatsEvaluator
+      extends GenericUDAFNumericStatsEvaluator<Double, DoubleObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.DoubleStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  @VectorizedUDAFs({
+      VectorUDAFComputeBitVectorDecimal.class,
+      VectorUDAFComputeBitVectorFinal.class
+  })
+  public static class GenericUDAFDecimalStatsEvaluator
+      extends GenericUDAFNumericStatsEvaluator<HiveDecimal, HiveDecimalObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.DecimalStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  @VectorizedUDAFs({
+      VectorUDAFComputeBitVectorLong.class,
+      VectorUDAFComputeBitVectorFinal.class
+  })
+  public static class GenericUDAFDateStatsEvaluator
+      extends GenericUDAFNumericStatsEvaluator<DateWritableV2, DateObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.DateStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  @VectorizedUDAFs({
+      VectorUDAFComputeBitVectorTimestamp.class,
+      VectorUDAFComputeBitVectorFinal.class
+  })
+  public static class GenericUDAFTimestampStatsEvaluator
+      extends GenericUDAFNumericStatsEvaluator<TimestampWritableV2, TimestampObjectInspector> {
+
+    @Override
+    public AggregationBuffer getNewAggregationBuffer() throws HiveException {
+      AggregationBuffer result = new GenericUDAFComputeBitVectorBase.TimestampStatsAgg();
+      reset(result);
+      return result;
+    }
+  }
+
+  @VectorizedUDAFs({
+      VectorUDAFComputeBitVectorString.class,
+      VectorUDAFComputeBitVectorFinal.class
+  })
+  public static class GenericUDAFStringStatsEvaluator extends StringStatsEvaluatorBase {
+
+    @Override
+    public ObjectInspector init(Mode m, ObjectInspector[] parameters) throws HiveException {
+      super.init(m, parameters);
+
+      // initialize input
+      if (mode == Mode.PARTIAL1 || mode == Mode.COMPLETE) {
+        inputOI = (PrimitiveObjectInspector) parameters[0];
+      } else {
+        ndvFieldOI = (BinaryObjectInspector) parameters[0];
+      }
+
+      // initialize output
+      if (mode == Mode.PARTIAL1 || mode == Mode.PARTIAL2) {
+        partialResult = new BytesWritable();
+        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+      } else {
+        result = new BytesWritable();
+        return PrimitiveObjectInspectorFactory.writableBinaryObjectInspector;
+      }
+    }
+
+    @Override
+    public void iterate(AggregationBuffer agg, Object[] parameters) throws HiveException {
+      Object p = parameters[0];
+      GenericUDAFComputeBitVectorBase.StringStatsAgg myagg = (GenericUDAFComputeBitVectorBase.StringStatsAgg) agg;
+
+      if (myagg.firstItem) {
+        myagg.numDV = HyperLogLog.builder().setSizeOptimized().build();
+        myagg.numDV.reset();
+        myagg.firstItem = false;
+      }
+
+      String v = PrimitiveObjectInspectorUtils.getString(p, inputOI);
+      if (v != null) {
+        // Add string value to NumDistinctValue Estimator
+        myagg.numDV.addToEstimator(v);
+      }
+    }
+  }
+}
diff --git a/ql/src/test/queries/clientpositive/acid_vectorization_original.q b/ql/src/test/queries/clientpositive/acid_vectorization_original.q
index c6d790d..d0e70f3 100644
--- a/ql/src/test/queries/clientpositive/acid_vectorization_original.q
+++ b/ql/src/test/queries/clientpositive/acid_vectorization_original.q
@@ -137,9 +137,17 @@ select ROW__ID, * from over10k_orc_bucketed where ROW__ID is null;
 
 -- select ROW__ID, * from over10k_orc_bucketed where ROW__ID is null;
 
+-- TODO: Remove this line after fixing HIVE-24351
+set hive.vectorized.execution.enabled=false;
+
 CREATE TABLE over10k_orc STORED AS ORC as select * from over10k_n2 where t between 3 and 4;
+
 -- Make sure there are multiple original files
 INSERT INTO over10k_orc select * from over10k_n2 where t between 3 and 4;
+
+-- TODO: Remove this line after fixing HIVE-24351
+set hive.vectorized.execution.enabled=true;
+
 alter table over10k_orc set TBLPROPERTIES ('transactional'='true');
 
 -- row id is projected but there are no delete deltas
diff --git a/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q b/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q
index 1e21c92..c8bfaad 100644
--- a/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q
+++ b/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q
@@ -28,6 +28,9 @@ select count(*) FROM tbl1_n7 a LEFT OUTER JOIN tbl2_n6 b ON a.key = b.key;
 
 insert overwrite table tbl2_n6 select * from src where key < 200;
 
+describe formatted tbl2_n6 key;
+select count (distinct key) from tbl2_n6;
+
 -- Since tbl2_n6 is the bigger table, tbl1_n7 Right Outer Join tbl2_n6 can be performed
 explain
 select count(*) FROM tbl1_n7 a RIGHT OUTER JOIN tbl2_n6 b ON a.key = b.key;
diff --git a/ql/src/test/queries/clientpositive/compute_bit_vector.q b/ql/src/test/queries/clientpositive/compute_bit_vector.q
new file mode 100644
index 0000000..a8eb894
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/compute_bit_vector.q
@@ -0,0 +1,20 @@
+--! qt:dataset:src
+--! qt:dataset:alltypesorc
+
+select hex(compute_bit_vector_hll(ctinyint)) from alltypesorc;
+select hex(compute_bit_vector_hll(csmallint)) from alltypesorc;
+select hex(compute_bit_vector_hll(cint)) from alltypesorc;
+select hex(compute_bit_vector_hll(cast (cbigint as decimal))) from alltypesorc;
+select hex(compute_bit_vector_hll(cfloat)) from alltypesorc;
+select hex(compute_bit_vector_hll(cdouble)) from alltypesorc;
+select hex(compute_bit_vector_hll(cstring1)) from alltypesorc;
+select hex(compute_bit_vector_hll(cstring2)) from alltypesorc;
+select hex(compute_bit_vector_hll(ctimestamp1)) from alltypesorc;
+select hex(compute_bit_vector_hll(cast (ctimestamp2 as date))) from alltypesorc;
+
+create table test_compute_bit_vector (val1 string, val2 string);
+insert overwrite table test_compute_bit_vector select a.value, b.value from src as a, src as b;
+
+-- hll of two columns must be equal to each other
+select hex(compute_bit_vector_hll(val1)) from test_compute_bit_vector;
+select hex(compute_bit_vector_hll(val2)) from test_compute_bit_vector;
\ No newline at end of file
diff --git a/ql/src/test/queries/clientpositive/hll.q b/ql/src/test/queries/clientpositive/hll.q
index b6b67e2..78434b3 100644
--- a/ql/src/test/queries/clientpositive/hll.q
+++ b/ql/src/test/queries/clientpositive/hll.q
@@ -1,5 +1,6 @@
 --! qt:dataset:src
 set hive.mapred.mode=nonstrict;
+set hive.vectorized.execution.enabled=false;
 
 create table n(key int);
 
diff --git a/ql/src/test/queries/clientpositive/insert0.q b/ql/src/test/queries/clientpositive/insert0.q
index 4d02a73..8e8cbfc 100644
--- a/ql/src/test/queries/clientpositive/insert0.q
+++ b/ql/src/test/queries/clientpositive/insert0.q
@@ -8,8 +8,13 @@ DROP TABLE ctas_part;
 
 CREATE TABLE insert_into1_n1 (key int, value string);
 
+-- TODO: Remove this line after fixing HIVE-24532
+set hive.vectorized.execution.enabled=false;
 INSERT OVERWRITE TABLE insert_into1_n1 SELECT * from src ORDER BY key LIMIT 10;
 
+-- TODO: Remove this line after fixing HIVE-24532
+set hive.vectorized.execution.enabled=true;
+
 select * from insert_into1_n1 order by key;
 
 INSERT INTO TABLE insert_into1_n1 SELECT * from src ORDER BY key DESC LIMIT 10;
@@ -27,9 +32,15 @@ set hive.exec.dynamic.partition=true;
 
 create table ctas_part (key int, value string) partitioned by (modkey bigint);
 
+-- TODO: Remove this line after fixing HIVE-24532
+set hive.vectorized.execution.enabled=false;
+
 insert overwrite table ctas_part partition (modkey) 
 select key, value, ceil(key / 100) from src where key is not null order by key limit 10;
 
+-- TODO: Remove this line after fixing HIVE-24532
+set hive.vectorized.execution.enabled=true;
+
 select * from ctas_part order by key;
 
 
diff --git a/ql/src/test/queries/clientpositive/vector_data_types.q b/ql/src/test/queries/clientpositive/vector_data_types.q
index 332da13..8069e8c 100644
--- a/ql/src/test/queries/clientpositive/vector_data_types.q
+++ b/ql/src/test/queries/clientpositive/vector_data_types.q
@@ -20,6 +20,9 @@ CREATE TABLE over1k_n8(t tinyint,
 ROW FORMAT DELIMITED FIELDS TERMINATED BY '|'
 STORED AS TEXTFILE;
 
+-- TODO: Remove this line after fixing HIVE-24351
+set hive.vectorized.execution.enabled=false;
+
 LOAD DATA LOCAL INPATH '../../data/files/over1k' OVERWRITE INTO TABLE over1k_n8;
 
 CREATE TABLE over1korc_n1(t tinyint,
diff --git a/ql/src/test/queries/clientpositive/vectorized_stats.q b/ql/src/test/queries/clientpositive/vectorized_stats.q
new file mode 100644
index 0000000..ca2ad6a
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/vectorized_stats.q
@@ -0,0 +1,92 @@
+--! qt:dataset:alltypesorc
+--! qt:dataset:lineitem
+
+set hive.stats.autogather=true;
+set hive.explain.user=false;
+set hive.vectorized.execution.enabled=true;
+
+-- string
+
+explain vectorization detail
+create table table_onestring as select cstring1 as val1 from alltypesorc;
+
+create table table_onestring as select cstring1 as val1 from alltypesorc;
+
+describe formatted table_onestring;
+
+describe formatted table_onestring val1;
+
+explain
+select distinct(val1) from table_onestring;
+
+-- long
+
+explain vectorization detail
+create table table_onebigint as select cbigint as val1 from alltypesorc;
+
+create table table_onebigint as select cbigint as val1 from alltypesorc;
+
+describe formatted table_onebigint;
+
+describe formatted table_onebigint val1;
+
+explain
+select distinct(val1) from table_onebigint;
+
+-- double
+
+explain vectorization detail
+create table table_onedouble as select cdouble as val1 from alltypesorc;
+
+create table table_onedouble as select cdouble as val1 from alltypesorc;
+
+describe formatted table_onedouble;
+
+describe formatted table_onedouble val1;
+
+explain
+select distinct(val1) from table_onedouble;
+
+-- timestamp
+
+explain vectorization detail
+create table table_onetimestamp as select ctimestamp1 as val1 from alltypesorc;
+
+create table table_onetimestamp as select ctimestamp1 as val1 from alltypesorc;
+
+describe formatted table_onetimestamp;
+
+describe formatted table_onetimestamp val1;
+
+explain
+select distinct(val1) from table_onetimestamp;
+
+--decimal
+
+explain vectorization detail
+create table table_onedecimal as select cast(cbigint as decimal(10,2)) as val1 from alltypesorc;
+
+create table table_onedecimal as select cast(cbigint as decimal(10,2)) as val1 from alltypesorc;
+
+describe formatted table_onedecimal;
+
+describe formatted table_onedecimal val1;
+
+explain
+select distinct(val1) from table_onedecimal;
+
+
+--decimal
+
+explain vectorization detail
+create table table_onedate as select cast(L_COMMITDATE as date) as val1 from lineitem;
+
+create table table_onedate as select cast(L_COMMITDATE as date) as val1 from lineitem;
+
+describe formatted table_onedate;
+
+describe formatted table_onedate val1;
+
+explain
+select distinct(val1) from table_onedate;
+
diff --git a/ql/src/test/results/clientnegative/fileformat_void_input.q.out b/ql/src/test/results/clientnegative/fileformat_void_input.q.out
index 42eb757..b629a5e 100644
--- a/ql/src/test/results/clientnegative/fileformat_void_input.q.out
+++ b/ql/src/test/results/clientnegative/fileformat_void_input.q.out
@@ -10,4 +10,4 @@ POSTHOOK: query: CREATE TABLE dest1(key INT, value STRING) STORED AS
 POSTHOOK: type: CREATETABLE
 POSTHOOK: Output: database:default
 POSTHOOK: Output: default@dest1
-FAILED: SemanticException 1:705 Input format must implement InputFormat. Error encountered near token 'dest1'
+FAILED: SemanticException 1:693 Input format must implement InputFormat. Error encountered near token 'dest1'
diff --git a/ql/src/test/results/clientnegative/masking_mv.q.out b/ql/src/test/results/clientnegative/masking_mv.q.out
index 801be5d..ac93247 100644
--- a/ql/src/test/results/clientnegative/masking_mv.q.out
+++ b/ql/src/test/results/clientnegative/masking_mv.q.out
@@ -62,7 +62,7 @@ STAGE PLANS:
                 outputColumnNames: col1
                 Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
-                  aggregations: min(col1), max(col1), count(1), count(col1), compute_bit_vector(col1, 'hll')
+                  aggregations: min(col1), max(col1), count(1), count(col1), compute_bit_vector_hll(col1)
                   minReductionHashAggr: 0.99
                   mode: hash
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4
@@ -72,9 +72,10 @@ STAGE PLANS:
                     sort order: 
                     Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary)
+      Execution mode: vectorized
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4
           Statistics: Num rows: 1 Data size: 168 Basic stats: COMPLETE Column stats: COMPLETE
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 10badf0..b9753e9 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
@@ -188,7 +188,7 @@ STAGE PLANS:
             outputColumnNames: key, value
             Statistics: Num rows: 550 Data size: 2200 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
-              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
               keys: '1' (type: string)
               minReductionHashAggr: 0.99
               mode: hash
@@ -298,7 +298,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
           keys: '1' (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
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 72905af..437776a 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
@@ -407,7 +407,7 @@ STAGE PLANS:
             outputColumnNames: key, value
             Statistics: Num rows: 3223 Data size: 610250 Basic stats: COMPLETE Column stats: NONE
             Group By Operator
-              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
               keys: '2' (type: string)
               minReductionHashAggr: 0.99
               mode: hash
@@ -518,7 +518,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
           keys: '2' (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
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 79d0e15..f49827d 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
@@ -662,7 +662,7 @@ STAGE PLANS:
                   outputColumnNames: k1, v1, k2, v2
                   Statistics: Num rows: 550 Data size: 52250 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
-                    aggregations: min(k1), max(k1), count(1), count(k1), compute_bit_vector(k1, 'hll'), max(length(v1)), avg(COALESCE(length(v1),0)), count(v1), compute_bit_vector(v1, 'hll'), min(k2), max(k2), count(k2), compute_bit_vector(k2, 'hll'), max(length(v2)), avg(COALESCE(length(v2),0)), count(v2), compute_bit_vector(v2, 'hll')
+                    aggregations: min(k1), max(k1), count(1), count(k1), compute_bit_vector_hll(k1), max(length(v1)), avg(COALESCE(length(v1),0)), count(v1), compute_bit_vector_hll(v1), min(k2), max(k2), count(k2), compute_bit_vector_hll(k2), max(length(v2)), avg(COALESCE(length(v2),0)), count(v2), compute_bit_vector_hll(v2)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
@@ -674,7 +674,7 @@ STAGE PLANS:
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigint), _col16 (type: binary)
       Reduce Operator Tree:
         Group By Operator
-          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+          aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
           Statistics: Num rows: 1 Data size: 792 Basic stats: COMPLETE Column stats: NONE
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 04018ed..5ddcec5 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
@@ -268,7 +268,7 @@ STAGE PLANS:
                       outputColumnNames: key, value1, value2
                       Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -455,7 +455,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
@@ -697,7 +697,7 @@ STAGE PLANS:
                       outputColumnNames: key, value1, value2
                       Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -884,7 +884,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
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 a1bd8ac..f6db3ec 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
@@ -252,7 +252,7 @@ STAGE PLANS:
                       outputColumnNames: key, value1, value2
                       Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -439,7 +439,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
@@ -681,7 +681,7 @@ STAGE PLANS:
                       outputColumnNames: key, value1, value2
                       Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -868,7 +868,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
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 ed4d1f8..1af3a40 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
@@ -252,7 +252,7 @@ STAGE PLANS:
                       outputColumnNames: key, value1, value2
                       Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -439,7 +439,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
@@ -681,7 +681,7 @@ STAGE PLANS:
                       outputColumnNames: key, value1, value2
                       Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                       Group By Operator
-                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -868,7 +868,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin5.q.out b/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
index dbdf1d7..b1b2e85 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin5.q.out
@@ -283,7 +283,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 312 Data size: 178025 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -388,7 +388,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
@@ -827,7 +827,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 163 Data size: 93968 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -932,7 +932,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out b/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
index 309eadc..2db2191 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin_negative.q.out
@@ -213,7 +213,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 146 Data size: 70215 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -227,6 +227,7 @@ STAGE PLANS:
                         tag: -1
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary)
                         auto parallelism: false
+      Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
       Path -> Alias:
@@ -272,7 +273,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out b/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
index 93980ab..8fc38cf 100644
--- a/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
+++ b/ql/src/test/results/clientpositive/bucketmapjoin_negative2.q.out
@@ -270,7 +270,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 163 Data size: 93968 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -284,6 +284,7 @@ STAGE PLANS:
                         tag: -1
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary)
                         auto parallelism: false
+      Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
       Path -> Alias:
@@ -329,7 +330,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
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 14d38db..30cdaa2 100644
--- a/ql/src/test/results/clientpositive/join_map_ppr.q.out
+++ b/ql/src/test/results/clientpositive/join_map_ppr.q.out
@@ -143,7 +143,7 @@ STAGE PLANS:
                     outputColumnNames: key, value, val2
                     Statistics: Num rows: 1100 Data size: 195800 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll'), max(length(val2)), avg(COALESCE(length(val2),0)), count(val2), compute_bit_vector(val2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value), max(length(val2)), avg(COALESCE(length(val2),0)), count(val2), compute_bit_vector_hll(val2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -157,6 +157,7 @@ STAGE PLANS:
                         tag: -1
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary)
                         auto parallelism: false
+      Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
       Path -> Alias:
@@ -204,7 +205,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: COMPLETE Column stats: NONE
@@ -730,7 +731,7 @@ STAGE PLANS:
                     outputColumnNames: key, value, val2
                     Statistics: Num rows: 1100 Data size: 104500 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll'), max(length(val2)), avg(COALESCE(length(val2),0)), count(val2), compute_bit_vector(val2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value), max(length(val2)), avg(COALESCE(length(val2),0)), count(val2), compute_bit_vector_hll(val2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -744,6 +745,7 @@ STAGE PLANS:
                         tag: -1
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary)
                         auto parallelism: false
+      Execution mode: vectorized
       Local Work:
         Map Reduce Local Work
       Path -> Alias:
@@ -791,7 +793,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
           Statistics: Num rows: 1 Data size: 704 Basic stats: COMPLETE Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out b/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out
index f851915..3831ebd 100644
--- a/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out
+++ b/ql/src/test/results/clientpositive/list_bucket_dml_8.q.out
@@ -84,7 +84,7 @@ STAGE PLANS:
                 outputColumnNames: key, value, ds, hr
                 Statistics: Num rows: 1000 Data size: 358000 Basic stats: COMPLETE Column stats: COMPLETE
                 Group By Operator
-                  aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                  aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                   keys: ds (type: string), hr (type: string)
                   minReductionHashAggr: 0.99
                   mode: hash
@@ -101,6 +101,7 @@ STAGE PLANS:
                     tag: -1
                     value expressions: _col2 (type: int), _col3 (type: struct<count:bigint,sum:double,input:int>), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: struct<count:bigint,sum:double,input:int>), _col9 (type: bigint), _col10 (type: binary)
                     auto parallelism: false
+      Execution mode: vectorized
       Path -> Alias:
 #### A masked pattern was here ####
       Path -> Partition:
@@ -184,7 +185,7 @@ STAGE PLANS:
       Needs Tagging: false
       Reduce Operator Tree:
         Group By Operator
-          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+          aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
           keys: KEY._col0 (type: string), KEY._col1 (type: string)
           mode: mergepartial
           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
diff --git a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
index c749496..95f5c24 100644
--- a/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_no_buckets.q.out
@@ -1859,13 +1859,37 @@ STAGE PLANS:
                       className: VectorFileSinkOperator
                       native: false
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
                 enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_bit_vector not supported
-                vectorized: false
+                reduceColumnNullOrder: zz
+                reduceColumnSortOrder: ++
+                allNative: false
+                usesVectorUDFAdaptor: true
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 11
+                    dataColumns: KEY._col0:string, KEY._col1:string, VALUE._col0:int, VALUE._col1:struct<count:bigint,sum:double,input:int>, VALUE._col2:bigint, VALUE._col3:bigint, VALUE._col4:binary, VALUE._col5:int, VALUE._col6:struct<count:bigint,sum:double,input:int>, VALUE._col7:bigint, VALUE._col8:binary
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
             Reduce Operator Tree:
+                Group By Vectorization:
+                    aggregators: VectorUDAFMaxLong(col 2:int) -> int, VectorUDAFAvgFinal(col 3:struct<count:bigint,sum:double,input:int>) -> double, VectorUDAFCountMerge(col 4:bigint) -> bigint, VectorUDAFCountMerge(col 5:bigint) -> bigint, VectorUDAFComputeBitVectorFinal(col 6:binary) -> binary, VectorUDAFMaxLong(col 7:int) -> int, VectorUDAFAvgFinal(col 8:struct<count:bigint,sum:double,input:int>) -> double, VectorUDAFCountMerge(col 9:bigint) -> bigint, VectorUDAFComputeBitVectorFinal( [...]
+                    className: VectorGroupByOperator
+                    groupByMode: MERGEPARTIAL
+                    keyExpressions: col 0:string, col 1:string
+                    native: false
+                    vectorProcessingMode: MERGE_PARTIAL
+                    projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
+                  Select Vectorization:
+                      className: VectorSelectOperator
+                      native: true
+                      projectedOutputColumnNums: [11, 13, 15, 16, 19, 6, 20, 22, 24, 25, 28, 10, 0, 1]
+                      selectExpressions: ConstantVectorExpression(val STRING) -> 11:string, VectorCoalesce(columns [2, 12])(children: col 2:int, ConstantVectorExpression(val 0) -> 12:int) -> 13:int, VectorCoalesce(columns [3, 14])(children: col 3:double, ConstantVectorExpression(val 0.0) -> 14:double) -> 15:double, LongColSubtractLongColumn(col 4:bigint, col 5:bigint) -> 16:bigint, VectorCoalesce(columns [17, 18])(children: VectorUDFAdaptor(ndv_compute_bit_vector(_col6)) -> 17:bigint, Co [...]
+                    File Sink Vectorization:
+                        className: VectorFileSinkOperator
+                        native: false
         Reducer 5 
             Execution mode: vectorized, llap
             Reduce Vectorization:
@@ -2722,21 +2746,78 @@ STAGE PLANS:
                       className: VectorFileSinkOperator
                       native: false
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
                 enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_bit_vector not supported
-                vectorized: false
+                reduceColumnNullOrder: a
+                reduceColumnSortOrder: +
+                allNative: false
+                usesVectorUDFAdaptor: false
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 4
+                    dataColumns: KEY.reducesinkkey0:string, VALUE._col0:string, VALUE._col1:string, VALUE._col2:string
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: [bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint, bigint]
             Reduce Operator Tree:
+                Select Vectorization:
+                    className: VectorSelectOperator
+                    native: true
+                    projectedOutputColumnNums: [0, 1, 2, 3]
+                  File Sink Vectorization:
+                      className: VectorFileSinkOperator
+                      native: false
+                  Select Vectorization:
+                      className: VectorSelectOperator
+                      native: true
+                      projectedOutputColumnNums: [0, 1, 2, 3]
+                    Group By Vectorization:
+                        aggregators: VectorUDAFMaxLong(StringLength(col 0:string) -> 4:int) -> int, VectorUDAFAvgLong(VectorCoalesce(columns [5, 6])(children: StringLength(col 0:string) -> 5:int, ConstantVectorExpression(val 0) -> 6:int) -> 7:int) -> struct<count:bigint,sum:double,input:int>, VectorUDAFCount(ConstantVectorExpression(val 1) -> 8:int) -> bigint, VectorUDAFCount(col 0:string) -> bigint, VectorUDAFComputeBitVectorString(col 0:string) -> binary, VectorUDAFMaxLong(StringLength [...]
+                        className: VectorGroupByOperator
+                        groupByMode: HASH
+                        keyExpressions: col 2:string, col 3:string
+                        native: false
+                        vectorProcessingMode: HASH
+                        projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
+                      Reduce Sink Vectorization:
+                          className: VectorReduceSinkMultiKeyOperator
+                          keyColumns: 0:string, 1:string
+                          native: true
+                          nativeConditionsMet: hive.vectorized.execution.reducesink.new.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true, No PTF TopN IS true, No DISTINCT columns IS true, BinarySortableSerDe for keys IS true, LazyBinarySerDe for values IS true
+                          valueColumns: 2:int, 3:struct<count:bigint,sum:double,input:int>, 4:bigint, 5:bigint, 6:binary, 7:int, 8:struct<count:bigint,sum:double,input:int>, 9:bigint, 10:binary
         Reducer 5 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Vectorization:
                 enabled: true
                 enableConditionsMet: hive.vectorized.execution.reduce.enabled IS true, hive.execution.engine tez IN [tez, spark] IS true
-                notVectorizedReason: Aggregation Function expression for GROUPBY operator: UDF compute_bit_vector not supported
-                vectorized: false
+                reduceColumnNullOrder: zz
+                reduceColumnSortOrder: ++
+                allNative: false
+                usesVectorUDFAdaptor: true
+                vectorized: true
+                rowBatchContext:
+                    dataColumnCount: 11
+                    dataColumns: KEY._col0:string, KEY._col1:string, VALUE._col0:int, VALUE._col1:struct<count:bigint,sum:double,input:int>, VALUE._col2:bigint, VALUE._col3:bigint, VALUE._col4:binary, VALUE._col5:int, VALUE._col6:struct<count:bigint,sum:double,input:int>, VALUE._col7:bigint, VALUE._col8:binary
+                    partitionColumnCount: 0
+                    scratchColumnTypeNames: []
             Reduce Operator Tree:
+                Group By Vectorization:
+                    aggregators: VectorUDAFMaxLong(col 2:int) -> int, VectorUDAFAvgFinal(col 3:struct<count:bigint,sum:double,input:int>) -> double, VectorUDAFCountMerge(col 4:bigint) -> bigint, VectorUDAFCountMerge(col 5:bigint) -> bigint, VectorUDAFComputeBitVectorFinal(col 6:binary) -> binary, VectorUDAFMaxLong(col 7:int) -> int, VectorUDAFAvgFinal(col 8:struct<count:bigint,sum:double,input:int>) -> double, VectorUDAFCountMerge(col 9:bigint) -> bigint, VectorUDAFComputeBitVectorFinal( [...]
+                    className: VectorGroupByOperator
+                    groupByMode: MERGEPARTIAL
+                    keyExpressions: col 0:string, col 1:string
+                    native: false
+                    vectorProcessingMode: MERGE_PARTIAL
+                    projectedOutputColumnNums: [0, 1, 2, 3, 4, 5, 6, 7, 8]
+                  Select Vectorization:
+                      className: VectorSelectOperator
+                      native: true
+                      projectedOutputColumnNums: [11, 13, 15, 16, 19, 6, 20, 22, 24, 25, 28, 10, 0, 1]
+                      selectExpressions: ConstantVectorExpression(val STRING) -> 11:string, VectorCoalesce(columns [2, 12])(children: col 2:int, ConstantVectorExpression(val 0) -> 12:int) -> 13:int, VectorCoalesce(columns [3, 14])(children: col 3:double, ConstantVectorExpression(val 0.0) -> 14:double) -> 15:double, LongColSubtractLongColumn(col 4:bigint, col 5:bigint) -> 16:bigint, VectorCoalesce(columns [17, 18])(children: VectorUDFAdaptor(ndv_compute_bit_vector(_col6)) -> 17:bigint, Co [...]
+                    File Sink Vectorization:
+                        className: VectorFileSinkOperator
+                        native: false
         Reducer 6 
             Execution mode: vectorized, llap
             Reduce Vectorization:
diff --git a/ql/src/test/results/clientpositive/llap/acid_stats5.q.out b/ql/src/test/results/clientpositive/llap/acid_stats5.q.out
index c899dbf..67030e9 100644
--- a/ql/src/test/results/clientpositive/llap/acid_stats5.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_stats5.q.out
@@ -895,7 +895,7 @@ avg_col_len
 max_col_len         	                    
 num_trues           	                    
 num_falses          	                    
-bit_vector          	                    
+bit_vector          	HL                  
 comment             	from deserializer   
 COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"key\":\"true\",\"value\":\"true\"}}
 PREHOOK: query: explain select count(*) from stats2
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_11.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_11.q.out
index 46e0fe0..2679d7a 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_11.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_11.q.out
@@ -58,7 +58,7 @@ STAGE PLANS:
                           outputColumnNames: a, b, d
                           Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(d), max(d), count(d), compute_bit_vector(d, 'hll')
+                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(b), max(b), count(b), compute_bit_vector_hll(b), min(d), max(d), count(d), compute_bit_vector_hll(d)
                             minReductionHashAggr: 0.4
                             mode: hash
                             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -71,10 +71,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 592 Basic stats: COMPLETE Column stats: COMPLETE
@@ -293,7 +293,7 @@ STAGE PLANS:
                     outputColumnNames: a, b, d
                     Statistics: Num rows: 5 Data size: 320 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(d), max(d), count(d), compute_bit_vector(d, 'hll')
+                      aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(b), max(b), count(b), compute_bit_vector_hll(b), min(d), max(d), count(d), compute_bit_vector_hll(d)
                       minReductionHashAggr: 0.8
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -306,10 +306,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 592 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_4.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_4.q.out
index b6474f6..c30b58f 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_4.q.out
@@ -110,7 +110,7 @@ STAGE PLANS:
                     Statistics: Num rows: 10 Data size: 1296 Basic stats: COMPLETE Column stats: COMPLETE
                     value expressions: _col1 (type: varchar(128))
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: varchar(128))
@@ -130,7 +130,7 @@ STAGE PLANS:
                   outputColumnNames: a, b
                   Statistics: Num rows: 10 Data size: 1296 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector(b, 'hll')
+                    aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector_hll(b)
                     minReductionHashAggr: 0.9
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -141,10 +141,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_5.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_5.q.out
index b57c4a3..8af751c 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_5.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_5.q.out
@@ -59,7 +59,7 @@ STAGE PLANS:
                           outputColumnNames: a, b, part
                           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector(b, 'hll')
+                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector_hll(b)
                             keys: part (type: int)
                             minReductionHashAggr: 0.4
                             mode: hash
@@ -78,7 +78,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -285,7 +285,7 @@ STAGE PLANS:
                           outputColumnNames: a, b, c, d, part
                           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector(b, 'hll'), min(c), max(c), count(c), compute_bit_vector(c, 'hll'), max(length(d)), avg(COALESCE(length(d),0)), count(d), compute_bit_vector(d, 'hll')
+                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector_hll(b), min(c), max(c), count(c), compute_bit_vector_hll(c), max(length(d)), avg(COALESCE(length(d),0)), count(d), compute_bit_vector_hll(d)
                             keys: part (type: int)
                             minReductionHashAggr: 0.4
                             mode: hash
@@ -304,7 +304,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -469,7 +469,7 @@ STAGE PLANS:
                           outputColumnNames: a, b, c, d, part
                           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector(b, 'hll'), min(c), max(c), count(c), compute_bit_vector(c, 'hll'), max(length(d)), avg(COALESCE(length(d),0)), count(d), compute_bit_vector(d, 'hll')
+                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector_hll(b), min(c), max(c), count(c), compute_bit_vector_hll(c), max(length(d)), avg(COALESCE(length(d),0)), count(d), compute_bit_vector_hll(d)
                             keys: part (type: int)
                             minReductionHashAggr: 0.4
                             mode: hash
@@ -488,7 +488,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_5a.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_5a.q.out
index fb7db48..1d846bf 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_5a.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_5a.q.out
@@ -83,7 +83,7 @@ STAGE PLANS:
                           outputColumnNames: a, b, part
                           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector(b, 'hll')
+                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector_hll(b)
                             keys: part (type: int)
                             minReductionHashAggr: 0.4
                             mode: hash
@@ -144,7 +144,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -342,7 +342,7 @@ STAGE PLANS:
                           outputColumnNames: a, b, part
                           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector(b, 'hll')
+                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector_hll(b)
                             keys: part (type: int)
                             minReductionHashAggr: 0.4
                             mode: hash
@@ -403,7 +403,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -539,7 +539,7 @@ STAGE PLANS:
                           outputColumnNames: a, b, part
                           Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector(b, 'hll')
+                            aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), max(length(b)), avg(COALESCE(length(b),0)), count(b), compute_bit_vector_hll(b)
                             keys: part (type: int)
                             minReductionHashAggr: 0.4
                             mode: hash
@@ -558,7 +558,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_6.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_6.q.out
index cba23dc..e6ea95c 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_6.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_6.q.out
@@ -63,7 +63,7 @@ STAGE PLANS:
                       outputColumnNames: key, value, one, two, three
                       Statistics: Num rows: 500 Data size: 274000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                        aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                         keys: one (type: string), two (type: string), three (type: string)
                         minReductionHashAggr: 0.4
                         mode: hash
@@ -76,7 +76,7 @@ STAGE PLANS:
                           Map-reduce partition columns: _col0 (type: string), _col1 (type: string), _col2 (type: string)
                           Statistics: Num rows: 500 Data size: 426500 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col3 (type: int), _col4 (type: int), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: binary), _col8 (type: int), _col9 (type: struct<count:bigint,sum:double,input:int>), _col10 (type: bigint), _col11 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: vectorized, llap
@@ -94,10 +94,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.orcfile_merge2a
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string), KEY._col1 (type: string), KEY._col2 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_7.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_7.q.out
index 2340c1e..274eeb8 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_7.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_7.q.out
@@ -108,10 +108,10 @@ STAGE PLANS:
                       Statistics: Num rows: 250 Data size: 87584 Basic stats: COMPLETE Column stats: NONE
                       value expressions: length(key) (type: int), COALESCE(length(key),0) (type: int), key (type: string), c1 (type: int), length(c2) (type: int), COALESCE(length(c2),0) (type: int), c2 (type: string)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(1), count(VALUE._col3), compute_bit_vector(VALUE._col3, 'hll'), min(VALUE._col5), max(VALUE._col5), count(VALUE._col5), compute_bit_vector(VALUE._col5, 'hll'), max(VALUE._col6), avg(VALUE._col7), count(VALUE._col8), compute_bit_vector(VALUE._col8, 'hll')
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(1), count(VALUE._col3), compute_bit_vector_hll(VALUE._col3), min(VALUE._col4), max(VALUE._col4), count(VALUE._col4), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col7)
                 mode: partial1
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
@@ -121,10 +121,10 @@ STAGE PLANS:
                   Statistics: Num rows: 1 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
                   value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary)
         Reducer 5 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: final
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 1000 Basic stats: COMPLETE Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_8.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_8.q.out
index d91f0ad..08d6db4 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_8.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_8.q.out
@@ -120,7 +120,7 @@ STAGE PLANS:
                         outputColumnNames: key, value, ds, hr
                         Statistics: Num rows: 666 Data size: 363636 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                          aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                           keys: ds (type: string), hr (type: string)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -179,7 +179,7 @@ STAGE PLANS:
                         outputColumnNames: key, value, hr
                         Statistics: Num rows: 666 Data size: 303696 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                          aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                           keys: '2008-12-31' (type: string), hr (type: string)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -196,7 +196,7 @@ STAGE PLANS:
                             tag: -1
                             value expressions: _col2 (type: int), _col3 (type: struct<count:bigint,sum:double,input:int>), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: struct<count:bigint,sum:double,input:int>), _col9 (type: bigint), _col10 (type: binary)
                             auto parallelism: true
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
             Path -> Alias:
 #### A masked pattern was here ####
@@ -355,11 +355,11 @@ STAGE PLANS:
               /srcpart/ds=2008-04-09/hr=11 [srcpart]
               /srcpart/ds=2008-04-09/hr=12 [srcpart]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -393,11 +393,11 @@ STAGE PLANS:
                     GatherStats: false
                     MultiFileSpray: false
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: '2008-12-31' (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
diff --git a/ql/src/test/results/clientpositive/llap/autoColumnStats_9.q.out b/ql/src/test/results/clientpositive/llap/autoColumnStats_9.q.out
index 8906791..0947cc9 100644
--- a/ql/src/test/results/clientpositive/llap/autoColumnStats_9.q.out
+++ b/ql/src/test/results/clientpositive/llap/autoColumnStats_9.q.out
@@ -104,7 +104,7 @@ STAGE PLANS:
                     outputColumnNames: key, value
                     Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -115,10 +115,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join1.q.out b/ql/src/test/results/clientpositive/llap/auto_join1.q.out
index 914b269..6d9611d 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join1.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join1.q.out
@@ -94,7 +94,7 @@ STAGE PLANS:
                             outputColumnNames: key, value
                             Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                               minReductionHashAggr: 0.99
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -104,13 +104,13 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join14.q.out b/ql/src/test/results/clientpositive/llap/auto_join14.q.out
index 72b9b48..2661303 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join14.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join14.q.out
@@ -100,7 +100,7 @@ STAGE PLANS:
                             outputColumnNames: c1, c2
                             Statistics: Num rows: 174 Data size: 16530 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector(c1, 'hll'), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector(c2, 'hll')
+                              aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector_hll(c1), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector_hll(c2)
                               minReductionHashAggr: 0.99
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -110,13 +110,13 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join17.q.out b/ql/src/test/results/clientpositive/llap/auto_join17.q.out
index 6a104ff..9c87725 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join17.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join17.q.out
@@ -73,7 +73,7 @@ STAGE PLANS:
                             outputColumnNames: key1, value1, key2, value2
                             Statistics: Num rows: 791 Data size: 150290 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(key1), max(key1), count(1), count(key1), compute_bit_vector(key1, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                              aggregations: min(key1), max(key1), count(1), count(key1), compute_bit_vector_hll(key1), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                               minReductionHashAggr: 0.99
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
@@ -83,7 +83,7 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigint), _col16 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 3 
             Map Operator Tree:
@@ -108,10 +108,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Statistics: Num rows: 1 Data size: 656 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join19.q.out b/ql/src/test/results/clientpositive/llap/auto_join19.q.out
index be3e6fd..ce46645 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join19.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join19.q.out
@@ -85,7 +85,7 @@ STAGE PLANS:
                             outputColumnNames: key, value
                             Statistics: Num rows: 3164 Data size: 300580 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                               minReductionHashAggr: 0.99
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -95,7 +95,7 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 3 
             Map Operator Tree:
@@ -120,10 +120,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join19_inclause.q.out b/ql/src/test/results/clientpositive/llap/auto_join19_inclause.q.out
index f2a7cb8..cf46435 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join19_inclause.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join19_inclause.q.out
@@ -85,7 +85,7 @@ STAGE PLANS:
                             outputColumnNames: key, value
                             Statistics: Num rows: 3164 Data size: 300580 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                               minReductionHashAggr: 0.99
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -95,7 +95,7 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 3 
             Map Operator Tree:
@@ -120,10 +120,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join2.q.out b/ql/src/test/results/clientpositive/llap/auto_join2.q.out
index 07b9b5d..04f4914 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join2.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join2.q.out
@@ -83,7 +83,7 @@ STAGE PLANS:
                               outputColumnNames: key, value
                               Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                               Group By Operator
-                                aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                                aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                                 minReductionHashAggr: 0.99
                                 mode: hash
                                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -93,7 +93,7 @@ STAGE PLANS:
                                   sort order: 
                                   Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                                   value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 3 
             Map Operator Tree:
@@ -140,10 +140,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join26.q.out b/ql/src/test/results/clientpositive/llap/auto_join26.q.out
index a597c2b..57f1ae2 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join26.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join26.q.out
@@ -97,7 +97,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0)
@@ -122,7 +122,7 @@ STAGE PLANS:
                     outputColumnNames: key, cnt
                     Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(cnt), max(cnt), count(cnt), compute_bit_vector(cnt, 'hll')
+                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(cnt), max(cnt), count(cnt), compute_bit_vector_hll(cnt)
                       minReductionHashAggr: 0.9375
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -133,10 +133,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join3.q.out b/ql/src/test/results/clientpositive/llap/auto_join3.q.out
index 87e735d..d25f17d 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join3.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join3.q.out
@@ -83,7 +83,7 @@ STAGE PLANS:
                               outputColumnNames: key, value
                               Statistics: Num rows: 1251 Data size: 118845 Basic stats: COMPLETE Column stats: COMPLETE
                               Group By Operator
-                                aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                                aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                                 minReductionHashAggr: 0.99
                                 mode: hash
                                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -93,7 +93,7 @@ STAGE PLANS:
                                   sort order: 
                                   Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                                   value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 3 
             Map Operator Tree:
@@ -139,10 +139,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join4.q.out b/ql/src/test/results/clientpositive/llap/auto_join4.q.out
index f7b9db7..034d5b3 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join4.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join4.q.out
@@ -95,7 +95,7 @@ STAGE PLANS:
                             outputColumnNames: c1, c2, c3, c4
                             Statistics: Num rows: 55 Data size: 10450 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector(c1, 'hll'), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector(c4, 'hll')
+                              aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector_hll(c1), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector_hll(c4)
                               minReductionHashAggr: 0.9818182
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
@@ -105,7 +105,7 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigint), _col16 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 3 
             Map Operator Tree:
@@ -130,10 +130,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Statistics: Num rows: 1 Data size: 656 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join5.q.out b/ql/src/test/results/clientpositive/llap/auto_join5.q.out
index 7bf4b0b..d2202ee 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join5.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join5.q.out
@@ -117,7 +117,7 @@ STAGE PLANS:
                             outputColumnNames: c1, c2, c3, c4
                             Statistics: Num rows: 55 Data size: 10450 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector(c1, 'hll'), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector(c4, 'hll')
+                              aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector_hll(c1), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector_hll(c4)
                               minReductionHashAggr: 0.9818182
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
@@ -127,13 +127,13 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigint), _col16 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Statistics: Num rows: 1 Data size: 656 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join6.q.out b/ql/src/test/results/clientpositive/llap/auto_join6.q.out
index 8575dbb..fcbf556 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join6.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join6.q.out
@@ -127,7 +127,7 @@ STAGE PLANS:
                     outputColumnNames: c1, c2, c3, c4
                     Statistics: Num rows: 110 Data size: 20900 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector(c1, 'hll'), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector(c4, 'hll')
+                      aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector_hll(c1), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector_hll(c4)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
@@ -138,10 +138,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigint), _col16 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Statistics: Num rows: 1 Data size: 656 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join7.q.out b/ql/src/test/results/clientpositive/llap/auto_join7.q.out
index 89a26a6..636e38f 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join7.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join7.q.out
@@ -169,7 +169,7 @@ STAGE PLANS:
                       outputColumnNames: c1, c2, c3, c4, c5, c6
                       Statistics: Num rows: 110 Data size: 31350 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector(c1, 'hll'), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector(c4, 'hll'), min(c5), max(c5), count(c5), compute_bit_vector(c5, 'hll'), max(length(c6)), avg(COALESCE(length(c6),0)), count(c6), compute_bit_vector(c6, 'hll')
+                        aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector_hll(c1), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector_hll(c4), min(c5), max(c5), count(c5), compute_bit_vector_hll(c5), max(length(c6)), avg(COALESCE(length(c6),0)), count(c6), compute_bit_vector_hll(c6)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24
@@ -180,10 +180,10 @@ STAGE PLANS:
                           Statistics: Num rows: 1 Data size: 1184 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigint), _col16 (type: binary), _col17 (type:  [...]
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16), min(VALUE._col17), max(VALUE._col18), count(VALUE._col19), compute_bit_vector(V [...]
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16), min(VALUE._col17), max(VALUE._col18), count(VALUE._col19), comp [...]
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24
                 Statistics: Num rows: 1 Data size: 980 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join8.q.out b/ql/src/test/results/clientpositive/llap/auto_join8.q.out
index 4caf63b..c901e02 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join8.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join8.q.out
@@ -98,7 +98,7 @@ STAGE PLANS:
                               outputColumnNames: c1, c2, c3, c4
                               Statistics: Num rows: 1 Data size: 190 Basic stats: COMPLETE Column stats: COMPLETE
                               Group By Operator
-                                aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector(c1, 'hll'), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector(c4, 'hll')
+                                aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector_hll(c1), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), max(length(c4)), avg(COALESCE(length(c4),0)), count(c4), compute_bit_vector_hll(c4)
                                 minReductionHashAggr: 0.4
                                 mode: hash
                                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
@@ -108,7 +108,7 @@ STAGE PLANS:
                                   sort order: 
                                   Statistics: Num rows: 1 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
                                   value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigint), _col16 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 3 
             Map Operator Tree:
@@ -133,10 +133,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Statistics: Num rows: 1 Data size: 656 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_join9.q.out b/ql/src/test/results/clientpositive/llap/auto_join9.q.out
index fa3e4e6..3856bd1 100644
--- a/ql/src/test/results/clientpositive/llap/auto_join9.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_join9.q.out
@@ -98,7 +98,7 @@ STAGE PLANS:
                             outputColumnNames: key, value
                             Statistics: Num rows: 791 Data size: 75145 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                               minReductionHashAggr: 0.99
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -108,13 +108,13 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 400 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_smb_mapjoin_14.q.out b/ql/src/test/results/clientpositive/llap/auto_smb_mapjoin_14.q.out
index 18fdebc..f6fdd59 100644
--- a/ql/src/test/results/clientpositive/llap/auto_smb_mapjoin_14.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_smb_mapjoin_14.q.out
@@ -1763,7 +1763,7 @@ STAGE PLANS:
                               outputColumnNames: key, value
                               Statistics: Num rows: 16 Data size: 1488 Basic stats: COMPLETE Column stats: COMPLETE
                               Group By Operator
-                                aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                                aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                                 minReductionHashAggr: 0.9375
                                 mode: hash
                                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -1786,7 +1786,7 @@ STAGE PLANS:
                             outputColumnNames: key, val1, val2
                             Statistics: Num rows: 16 Data size: 2912 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(val1)), avg(COALESCE(length(val1),0)), count(val1), compute_bit_vector(val1, 'hll'), max(length(val2)), avg(COALESCE(length(val2),0)), count(val2), compute_bit_vector(val2, 'hll')
+                              aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(val1)), avg(COALESCE(length(val1),0)), count(val1), compute_bit_vector_hll(val1), max(length(val2)), avg(COALESCE(length(val2),0)), count(val2), compute_bit_vector_hll(val2)
                               minReductionHashAggr: 0.9375
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -1798,10 +1798,10 @@ STAGE PLANS:
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary)
             Execution mode: llap
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1817,10 +1817,10 @@ STAGE PLANS:
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 496 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2063,7 +2063,7 @@ STAGE PLANS:
                           outputColumnNames: key, value
                           Statistics: Num rows: 16 Data size: 1488 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                            aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                             minReductionHashAggr: 0.9375
                             mode: hash
                             outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -2093,10 +2093,10 @@ STAGE PLANS:
                               value expressions: _col1 (type: bigint)
             Execution mode: llap
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
@@ -2112,7 +2112,7 @@ STAGE PLANS:
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0)
@@ -2137,7 +2137,7 @@ STAGE PLANS:
                     outputColumnNames: key, cnt
                     Statistics: Num rows: 6 Data size: 48 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(cnt), max(cnt), count(cnt), compute_bit_vector(cnt, 'hll')
+                      aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(cnt), max(cnt), count(cnt), compute_bit_vector_hll(cnt)
                       minReductionHashAggr: 0.8333333
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -2148,10 +2148,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_13.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_13.q.out
index b7b9c5e..cf1a085 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_13.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_13.q.out
@@ -133,7 +133,7 @@ STAGE PLANS:
                             outputColumnNames: k1, k2
                             Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(k1), max(k1), count(1), count(k1), compute_bit_vector(k1, 'hll'), min(k2), max(k2), count(k2), compute_bit_vector(k2, 'hll')
+                              aggregations: min(k1), max(k1), count(1), count(k1), compute_bit_vector_hll(k1), min(k2), max(k2), count(k2), compute_bit_vector_hll(k2)
                               minReductionHashAggr: 0.9375
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -160,7 +160,7 @@ STAGE PLANS:
                             outputColumnNames: k1, k2
                             Statistics: Num rows: 16 Data size: 2848 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: max(length(k1)), avg(COALESCE(length(k1),0)), count(1), count(k1), compute_bit_vector(k1, 'hll'), max(length(k2)), avg(COALESCE(length(k2),0)), count(k2), compute_bit_vector(k2, 'hll')
+                              aggregations: max(length(k1)), avg(COALESCE(length(k1),0)), count(1), count(k1), compute_bit_vector_hll(k1), max(length(k2)), avg(COALESCE(length(k2),0)), count(k2), compute_bit_vector_hll(k2)
                               minReductionHashAggr: 0.9375
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -170,7 +170,7 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 4 
             Map Operator Tree:
@@ -195,10 +195,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
@@ -214,10 +214,10 @@ STAGE PLANS:
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
@@ -441,7 +441,7 @@ STAGE PLANS:
                             outputColumnNames: k1, k2
                             Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(k1), max(k1), count(1), count(k1), compute_bit_vector(k1, 'hll'), min(k2), max(k2), count(k2), compute_bit_vector(k2, 'hll')
+                              aggregations: min(k1), max(k1), count(1), count(k1), compute_bit_vector_hll(k1), min(k2), max(k2), count(k2), compute_bit_vector_hll(k2)
                               minReductionHashAggr: 0.9375
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -468,7 +468,7 @@ STAGE PLANS:
                             outputColumnNames: k1, k2
                             Statistics: Num rows: 16 Data size: 2848 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: max(length(k1)), avg(COALESCE(length(k1),0)), count(1), count(k1), compute_bit_vector(k1, 'hll'), max(length(k2)), avg(COALESCE(length(k2),0)), count(k2), compute_bit_vector(k2, 'hll')
+                              aggregations: max(length(k1)), avg(COALESCE(length(k1),0)), count(1), count(k1), compute_bit_vector_hll(k1), max(length(k2)), avg(COALESCE(length(k2),0)), count(k2), compute_bit_vector_hll(k2)
                               minReductionHashAggr: 0.9375
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -478,7 +478,7 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 4 
             Map Operator Tree:
@@ -503,10 +503,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
@@ -522,10 +522,10 @@ STAGE PLANS:
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
@@ -749,7 +749,7 @@ STAGE PLANS:
                             outputColumnNames: k1, k2
                             Statistics: Num rows: 16 Data size: 128 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(k1), max(k1), count(1), count(k1), compute_bit_vector(k1, 'hll'), min(k2), max(k2), count(k2), compute_bit_vector(k2, 'hll')
+                              aggregations: min(k1), max(k1), count(1), count(k1), compute_bit_vector_hll(k1), min(k2), max(k2), count(k2), compute_bit_vector_hll(k2)
                               minReductionHashAggr: 0.9375
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -776,7 +776,7 @@ STAGE PLANS:
                             outputColumnNames: k1, k2
                             Statistics: Num rows: 16 Data size: 2848 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: max(length(k1)), avg(COALESCE(length(k1),0)), count(1), count(k1), compute_bit_vector(k1, 'hll'), max(length(k2)), avg(COALESCE(length(k2),0)), count(k2), compute_bit_vector(k2, 'hll')
+                              aggregations: max(length(k1)), avg(COALESCE(length(k1),0)), count(1), count(k1), compute_bit_vector_hll(k1), max(length(k2)), avg(COALESCE(length(k2),0)), count(k2), compute_bit_vector_hll(k2)
                               minReductionHashAggr: 0.9375
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -786,7 +786,7 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 4 
             Map Operator Tree:
@@ -811,10 +811,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: COMPLETE
@@ -830,10 +830,10 @@ STAGE PLANS:
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_14.q.out b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_14.q.out
index dfbf8a6..8b003f8 100644
--- a/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_14.q.out
+++ b/ql/src/test/results/clientpositive/llap/auto_sortmerge_join_14.q.out
@@ -142,6 +142,34 @@ POSTHOOK: Input: default@src
 POSTHOOK: Output: default@tbl2_n6
 POSTHOOK: Lineage: tbl2_n6.key EXPRESSION [(src)src.FieldSchema(name:key, type:string, comment:default), ]
 POSTHOOK: Lineage: tbl2_n6.value SIMPLE [(src)src.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: describe formatted tbl2_n6 key
+PREHOOK: type: DESCTABLE
+PREHOOK: Input: default@tbl2_n6
+POSTHOOK: query: describe formatted tbl2_n6 key
+POSTHOOK: type: DESCTABLE
+POSTHOOK: Input: default@tbl2_n6
+col_name            	key                 
+data_type           	int                 
+min                 	0                   
+max                 	199                 
+num_nulls           	0                   
+distinct_count      	121                 
+avg_col_len         	                    
+max_col_len         	                    
+num_trues           	                    
+num_falses          	                    
+bit_vector          	HL                  
+comment             	from deserializer   
+COLUMN_STATS_ACCURATE	{\"BASIC_STATS\":\"true\",\"COLUMN_STATS\":{\"key\":\"true\",\"value\":\"true\"}}
+PREHOOK: query: select count (distinct key) from tbl2_n6
+PREHOOK: type: QUERY
+PREHOOK: Input: default@tbl2_n6
+#### A masked pattern was here ####
+POSTHOOK: query: select count (distinct key) from tbl2_n6
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@tbl2_n6
+#### A masked pattern was here ####
+121
 PREHOOK: query: explain
 select count(*) FROM tbl1_n7 a RIGHT OUTER JOIN tbl2_n6 b ON a.key = b.key
 PREHOOK: type: QUERY
@@ -194,7 +222,7 @@ STAGE PLANS:
                       keys:
                         0 _col0 (type: int)
                         1 _col0 (type: int)
-                      Statistics: Num rows: 221 Data size: 1768 Basic stats: COMPLETE Column stats: COMPLETE
+                      Statistics: Num rows: 220 Data size: 1760 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
                         aggregations: count()
                         minReductionHashAggr: 0.99
diff --git a/ql/src/test/results/clientpositive/llap/binary_output_format.q.out b/ql/src/test/results/clientpositive/llap/binary_output_format.q.out
index f9c7900..dce0fd2 100644
--- a/ql/src/test/results/clientpositive/llap/binary_output_format.q.out
+++ b/ql/src/test/results/clientpositive/llap/binary_output_format.q.out
@@ -124,7 +124,7 @@ STAGE PLANS:
                         outputColumnNames: mydata
                         Statistics: Num rows: 500 Data size: 43500 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: max(length(mydata)), avg(COALESCE(length(mydata),0)), count(1), count(mydata), compute_bit_vector(mydata, 'hll')
+                          aggregations: max(length(mydata)), avg(COALESCE(length(mydata),0)), count(1), count(mydata), compute_bit_vector_hll(mydata)
                           minReductionHashAggr: 0.99
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4
@@ -178,11 +178,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /src [src]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1 Data size: 172 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/bucket1.q.out b/ql/src/test/results/clientpositive/llap/bucket1.q.out
index 601d776..216d900 100644
--- a/ql/src/test/results/clientpositive/llap/bucket1.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket1.q.out
@@ -96,7 +96,7 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /src [src]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
@@ -136,7 +136,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -151,11 +151,11 @@ STAGE PLANS:
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
                       auto parallelism: false
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/bucket2.q.out b/ql/src/test/results/clientpositive/llap/bucket2.q.out
index bf86af5e..732bfc6 100644
--- a/ql/src/test/results/clientpositive/llap/bucket2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket2.q.out
@@ -95,7 +95,7 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /src [src]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
@@ -135,7 +135,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                     Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/bucket3.q.out b/ql/src/test/results/clientpositive/llap/bucket3.q.out
index aa78675..940cd88 100644
--- a/ql/src/test/results/clientpositive/llap/bucket3.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket3.q.out
@@ -96,7 +96,7 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /src [src]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
@@ -139,7 +139,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 500 Data size: 90000 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.99
                     mode: hash
@@ -157,11 +157,11 @@ STAGE PLANS:
                       value expressions: _col1 (type: int), _col2 (type: int), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: struct<count:bigint,sum:double,input:int>), _col8 (type: bigint), _col9 (type: binary)
                       auto parallelism: true
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/bucket4.q.out b/ql/src/test/results/clientpositive/llap/bucket4.q.out
index 9329628..b0e81c6 100644
--- a/ql/src/test/results/clientpositive/llap/bucket4.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket4.q.out
@@ -95,7 +95,7 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /src [src]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
@@ -136,7 +136,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                     Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/bucket5.q.out b/ql/src/test/results/clientpositive/llap/bucket5.q.out
index 73301a3..4a8392c 100644
--- a/ql/src/test/results/clientpositive/llap/bucket5.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket5.q.out
@@ -132,7 +132,7 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /src [src]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
@@ -174,7 +174,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -189,11 +189,11 @@ STAGE PLANS:
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
                       auto parallelism: false
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
@@ -226,7 +226,7 @@ STAGE PLANS:
                     GatherStats: false
                     MultiFileSpray: false
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
@@ -265,7 +265,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -280,11 +280,11 @@ STAGE PLANS:
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
                       auto parallelism: false
         Reducer 5 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/bucket6.q.out b/ql/src/test/results/clientpositive/llap/bucket6.q.out
index f65ba13..60a9a94 100644
--- a/ql/src/test/results/clientpositive/llap/bucket6.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket6.q.out
@@ -58,7 +58,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
@@ -77,7 +77,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 2000 Data size: 356000 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -88,10 +88,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/bucket_many.q.out b/ql/src/test/results/clientpositive/llap/bucket_many.q.out
index 192005c..8ce661c 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_many.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_many.q.out
@@ -96,7 +96,7 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /src [src]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
@@ -136,7 +136,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -151,11 +151,11 @@ STAGE PLANS:
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
                       auto parallelism: false
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/bucket_num_reducers2.q.out b/ql/src/test/results/clientpositive/llap/bucket_num_reducers2.q.out
index 50e95ab..53ad0b4 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_num_reducers2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_num_reducers2.q.out
@@ -133,7 +133,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                     Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/bucketmapjoin1.q.out b/ql/src/test/results/clientpositive/llap/bucketmapjoin1.q.out
index ac906ab..bf1147c 100644
--- a/ql/src/test/results/clientpositive/llap/bucketmapjoin1.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketmapjoin1.q.out
@@ -649,7 +649,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -668,7 +668,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
@@ -1067,7 +1067,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -1086,7 +1086,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/bucketmapjoin2.q.out b/ql/src/test/results/clientpositive/llap/bucketmapjoin2.q.out
index 6473a52..7f64253 100644
--- a/ql/src/test/results/clientpositive/llap/bucketmapjoin2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketmapjoin2.q.out
@@ -326,7 +326,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -345,7 +345,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
@@ -755,7 +755,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -774,7 +774,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
@@ -1244,7 +1244,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 139 Data size: 21549 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -1263,7 +1263,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/bucketmapjoin3.q.out b/ql/src/test/results/clientpositive/llap/bucketmapjoin3.q.out
index 0245584..fe91a7f 100644
--- a/ql/src/test/results/clientpositive/llap/bucketmapjoin3.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketmapjoin3.q.out
@@ -350,7 +350,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -369,7 +369,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
@@ -779,7 +779,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 134 Data size: 20826 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -798,7 +798,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: PARTIAL Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/bucketmapjoin4.q.out b/ql/src/test/results/clientpositive/llap/bucketmapjoin4.q.out
index 587daf7..d319bd5 100644
--- a/ql/src/test/results/clientpositive/llap/bucketmapjoin4.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketmapjoin4.q.out
@@ -334,7 +334,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -353,7 +353,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: COMPLETE Column stats: NONE
@@ -735,7 +735,7 @@ STAGE PLANS:
                     outputColumnNames: key, value1, value2
                     Statistics: Num rows: 1 Data size: 206 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -754,7 +754,7 @@ STAGE PLANS:
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 704 Basic stats: COMPLETE Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_1.q.out b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_1.q.out
index 7a725c6..f0ca32d 100644
--- a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_1.q.out
@@ -431,7 +431,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 500 Data size: 90000 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.99
                     mode: hash
@@ -448,7 +448,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_2.q.out b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_2.q.out
index 33f0287..fe330fb 100644
--- a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_2.q.out
@@ -179,7 +179,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 14 Data size: 3822 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -196,7 +196,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -405,7 +405,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 29 Data size: 7917 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9655172
                     mode: hash
@@ -422,7 +422,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -655,7 +655,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 29 Data size: 7917 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9655172
                     mode: hash
@@ -672,7 +672,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -909,7 +909,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 14 Data size: 3822 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -926,7 +926,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -1145,7 +1145,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 14 Data size: 3822 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -1162,7 +1162,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -1381,7 +1381,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 14 Data size: 3822 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -1398,7 +1398,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_3.q.out b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_3.q.out
index c8a43e1..7e9fc0b 100644
--- a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_3.q.out
@@ -228,7 +228,7 @@ STAGE PLANS:
                   outputColumnNames: value, key, ds
                   Statistics: Num rows: 500 Data size: 136500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: max(length(value)), avg(COALESCE(length(value),0)), count(1), count(value), compute_bit_vector(value, 'hll'), min(key), max(key), count(key), compute_bit_vector(key, 'hll')
+                    aggregations: max(length(value)), avg(COALESCE(length(value),0)), count(1), count(value), compute_bit_vector_hll(value), min(key), max(key), count(key), compute_bit_vector_hll(key)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.99
                     mode: hash
@@ -245,7 +245,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_4.q.out b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_4.q.out
index 312b7e7..d6f5ce5 100644
--- a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_4.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_4.q.out
@@ -169,7 +169,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -186,7 +186,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -422,7 +422,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 14 Data size: 2492 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -439,7 +439,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_5.q.out b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_5.q.out
index bf5bb73..0092a86 100644
--- a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_5.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_5.q.out
@@ -169,7 +169,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 14 Data size: 3822 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -186,7 +186,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -413,7 +413,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 14 Data size: 3822 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -430,7 +430,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_6.q.out b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_6.q.out
index a26376b..5e3c429 100644
--- a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_6.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_6.q.out
@@ -170,7 +170,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -187,7 +187,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -414,7 +414,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -431,7 +431,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -658,7 +658,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -675,7 +675,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -837,7 +837,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -854,7 +854,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -1022,7 +1022,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -1039,7 +1039,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -1284,7 +1284,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -1301,7 +1301,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -1556,7 +1556,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -1573,7 +1573,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
diff --git a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_7.q.out b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_7.q.out
index 7ee12f2..ad89c4e 100644
--- a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_7.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_7.q.out
@@ -157,7 +157,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 3 Data size: 819 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.6666666
                     mode: hash
@@ -174,7 +174,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -385,7 +385,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 3 Data size: 819 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.6666666
                     mode: hash
@@ -402,7 +402,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -619,7 +619,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds
                   Statistics: Num rows: 3 Data size: 819 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.6666666
                     mode: hash
@@ -636,7 +636,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_8.q.out b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_8.q.out
index b881cfa..bdedb5d 100644
--- a/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_8.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucketsortoptimize_insert_8.q.out
@@ -169,7 +169,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -186,7 +186,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
@@ -408,7 +408,7 @@ STAGE PLANS:
                   outputColumnNames: key, key2, value, ds
                   Statistics: Num rows: 14 Data size: 3878 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string)
                     minReductionHashAggr: 0.9285714
                     mode: hash
@@ -425,7 +425,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13
diff --git a/ql/src/test/results/clientpositive/llap/case_sensitivity.q.out b/ql/src/test/results/clientpositive/llap/case_sensitivity.q.out
index 05d27c6..0a748bd 100644
--- a/ql/src/test/results/clientpositive/llap/case_sensitivity.q.out
+++ b/ql/src/test/results/clientpositive/llap/case_sensitivity.q.out
@@ -58,7 +58,7 @@ STAGE PLANS:
                         outputColumnNames: key, value
                         Statistics: Num rows: 3 Data size: 8039 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                          aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                           minReductionHashAggr: 0.99
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -68,13 +68,13 @@ STAGE PLANS:
                             sort order: 
                             Statistics: Num rows: 1 Data size: 3080 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: no inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 3080 Basic stats: COMPLETE Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/cast1.q.out b/ql/src/test/results/clientpositive/llap/cast1.q.out
index c623483..5defe53 100644
--- a/ql/src/test/results/clientpositive/llap/cast1.q.out
+++ b/ql/src/test/results/clientpositive/llap/cast1.q.out
@@ -56,7 +56,7 @@ STAGE PLANS:
                         outputColumnNames: c1, c2, c3, c4, c5, c6, c7
                         Statistics: Num rows: 250 Data size: 31000 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector(c1, 'hll'), min(c2), max(c2), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), min(c4), max(c4), count(c4), compute_bit_vector(c4, 'hll'), min(c5), max(c5), count(c5), compute_bit_vector(c5, 'hll'), max(length(c6)), avg(COALESCE(length(c6),0)), count(c6), compute_bit_vector(c6, 'hll'), min(c7), max(c7), count(c7), compute_bit_vector(c7, 'hll')
+                          aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector_hll(c1), min(c2), max(c2), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), min(c4), max(c4), count(c4), compute_bit_vector_hll(c4), min(c5), max(c5), count(c5), compute_bit_vector_hll(c5), max(length(c6)), avg(COALESCE(length(c6),0)), count(c6), compute_bit_vector_hll(c6), min(c7), max(c7), count(c7), compute_bit_vector_hll(c7)
                           minReductionHashAggr: 0.99
                           mode: hash
                           outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28
@@ -66,13 +66,13 @@ STAGE PLANS:
                             sort order: 
                             Statistics: Num rows: 1 Data size: 1224 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: double), _col6 (type: double), _col7 (type: bigint), _col8 (type: binary), _col9 (type: double), _col10 (type: double), _col11 (type: bigint), _col12 (type: binary), _col13 (type: double), _col14 (type: double), _col15 (type: bigint), _col16 (type: binary), _col17 (type: int), _col18 (type: int), _col19 (type: bigint), _col20  [...]
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16), min(VALUE._col17), max(VALUE._col18), count(VALUE._col19), compute_bit_vector(V [...]
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16), min(VALUE._col17), max(VALUE._col18), count(VALUE._col19), comp [...]
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28
                 Statistics: Num rows: 1 Data size: 1156 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/cbo_rp_auto_join17.q.out b/ql/src/test/results/clientpositive/llap/cbo_rp_auto_join17.q.out
index f91c795..457e6cb 100644
--- a/ql/src/test/results/clientpositive/llap/cbo_rp_auto_join17.q.out
+++ b/ql/src/test/results/clientpositive/llap/cbo_rp_auto_join17.q.out
@@ -73,7 +73,7 @@ STAGE PLANS:
                             outputColumnNames: key1, value1, key2, value2
                             Statistics: Num rows: 791 Data size: 150290 Basic stats: COMPLETE Column stats: COMPLETE
                             Group By Operator
-                              aggregations: min(key1), max(key1), count(1), count(key1), compute_bit_vector(key1, 'hll'), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector(value1, 'hll'), min(key2), max(key2), count(key2), compute_bit_vector(key2, 'hll'), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector(value2, 'hll')
+                              aggregations: min(key1), max(key1), count(1), count(key1), compute_bit_vector_hll(key1), max(length(value1)), avg(COALESCE(length(value1),0)), count(value1), compute_bit_vector_hll(value1), min(key2), max(key2), count(key2), compute_bit_vector_hll(key2), max(length(value2)), avg(COALESCE(length(value2),0)), count(value2), compute_bit_vector_hll(value2)
                               minReductionHashAggr: 0.99
                               mode: hash
                               outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
@@ -83,7 +83,7 @@ STAGE PLANS:
                                 sort order: 
                                 Statistics: Num rows: 1 Data size: 792 Basic stats: COMPLETE Column stats: COMPLETE
                                 value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigint), _col16 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Map 3 
             Map Operator Tree:
@@ -108,10 +108,10 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Statistics: Num rows: 1 Data size: 656 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/cbo_rp_gby2_map_multi_distinct.q.out b/ql/src/test/results/clientpositive/llap/cbo_rp_gby2_map_multi_distinct.q.out
index 7442736..d4cc601 100644
--- a/ql/src/test/results/clientpositive/llap/cbo_rp_gby2_map_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/llap/cbo_rp_gby2_map_multi_distinct.q.out
@@ -88,7 +88,7 @@ STAGE PLANS:
                     outputColumnNames: key, c1, c2, c3, c4
                     Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), min(c1), max(c1), count(c1), compute_bit_vector(c1, 'hll'), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), min(c4), max(c4), count(c4), compute_bit_vector(c4, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), min(c1), max(c1), count(c1), compute_bit_vector_hll(c1), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), min(c4), max(c4), count(c4), compute_bit_vector_hll(c4)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
@@ -99,10 +99,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 952 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: int), _col15 (type: bigint), _col16 (type: binary), _col17 (type: in [...]
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16), min(VALUE._col17), max(VALUE._col18), count(VALUE._col19), compute_bit_vector(V [...]
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16), min(VALUE._col17), max(VALUE._col18), count(VALUE._col19), comp [...]
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                 Statistics: Num rows: 1 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
@@ -258,7 +258,7 @@ STAGE PLANS:
                     outputColumnNames: key, c1, c2, c3, c4
                     Statistics: Num rows: 250 Data size: 70250 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), min(c1), max(c1), count(c1), compute_bit_vector(c1, 'hll'), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), min(c4), max(c4), count(c4), compute_bit_vector(c4, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), min(c1), max(c1), count(c1), compute_bit_vector_hll(c1), max(length(c2)), avg(COALESCE(length(c2),0)), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), min(c4), max(c4), count(c4), compute_bit_vector_hll(c4)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
@@ -269,10 +269,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 952 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: int), _col15 (type: bigint), _col16 (type: binary), _col17 (type: in [...]
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16), min(VALUE._col17), max(VALUE._col18), count(VALUE._col19), compute_bit_vector(V [...]
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16), min(VALUE._col17), max(VALUE._col18), count(VALUE._col19), comp [...]
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20
                 Statistics: Num rows: 1 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/cbo_rp_groupby3_noskew_multi_distinct.q.out b/ql/src/test/results/clientpositive/llap/cbo_rp_groupby3_noskew_multi_distinct.q.out
index 795d25d..fa5b466 100644
--- a/ql/src/test/results/clientpositive/llap/cbo_rp_groupby3_noskew_multi_distinct.q.out
+++ b/ql/src/test/results/clientpositive/llap/cbo_rp_groupby3_noskew_multi_distinct.q.out
@@ -96,7 +96,7 @@ STAGE PLANS:
                     outputColumnNames: c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11
                     Statistics: Num rows: 1 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector(c1, 'hll'), min(c2), max(c2), count(c2), compute_bit_vector(c2, 'hll'), min(c3), max(c3), count(c3), compute_bit_vector(c3, 'hll'), min(c4), max(c4), count(c4), compute_bit_vector(c4, 'hll'), min(c5), max(c5), count(c5), compute_bit_vector(c5, 'hll'), min(c6), max(c6), count(c6), compute_bit_vector(c6, 'hll'), min(c7), max(c7), count(c7), compute_bit_vector(c7, 'hll'), min(c8), max(c8), count(c8 [...]
+                      aggregations: min(c1), max(c1), count(1), count(c1), compute_bit_vector_hll(c1), min(c2), max(c2), count(c2), compute_bit_vector_hll(c2), min(c3), max(c3), count(c3), compute_bit_vector_hll(c3), min(c4), max(c4), count(c4), compute_bit_vector_hll(c4), min(c5), max(c5), count(c5), compute_bit_vector_hll(c5), min(c6), max(c6), count(c6), compute_bit_vector_hll(c6), min(c7), max(c7), count(c7), compute_bit_vector_hll(c7), min(c8), max(c8), count(c8), compute_bit_vector [...]
                       mode: complete
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44
                       Statistics: Num rows: 1 Data size: 1856 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/clusterctas.q.out b/ql/src/test/results/clientpositive/llap/clusterctas.q.out
index b919b47..4edd8af 100644
--- a/ql/src/test/results/clientpositive/llap/clusterctas.q.out
+++ b/ql/src/test/results/clientpositive/llap/clusterctas.q.out
@@ -47,7 +47,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string)
@@ -67,7 +67,7 @@ STAGE PLANS:
                   outputColumnNames: col1, col2
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                    aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -78,10 +78,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/column_pruner_multiple_children.q.out b/ql/src/test/results/clientpositive/llap/column_pruner_multiple_children.q.out
index c2d6074..d67da55 100644
--- a/ql/src/test/results/clientpositive/llap/column_pruner_multiple_children.q.out
+++ b/ql/src/test/results/clientpositive/llap/column_pruner_multiple_children.q.out
@@ -92,10 +92,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: key (type: int), length(value) (type: int), COALESCE(length(value),0) (type: int), value (type: string)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col0), count(1), count(VALUE._col0), compute_bit_vector(VALUE._col0, 'hll'), max(VALUE._col3), avg(VALUE._col4), count(VALUE._col5), compute_bit_vector(VALUE._col5, 'hll')
+                aggregations: min(VALUE._col0), max(VALUE._col0), count(1), count(VALUE._col0), compute_bit_vector_hll(VALUE._col0), max(VALUE._col2), avg(VALUE._col3), count(VALUE._col4), compute_bit_vector_hll(VALUE._col4)
                 mode: complete
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/column_table_stats.q.out b/ql/src/test/results/clientpositive/llap/column_table_stats.q.out
index 3aeeefa..e2f0fd9 100644
--- a/ql/src/test/results/clientpositive/llap/column_table_stats.q.out
+++ b/ql/src/test/results/clientpositive/llap/column_table_stats.q.out
@@ -86,7 +86,7 @@ STAGE PLANS:
                     outputColumnNames: key, value
                     Statistics: Num rows: 1 Data size: 368 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -140,11 +140,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /s [s]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 840 Basic stats: COMPLETE Column stats: NONE
@@ -352,7 +352,7 @@ STAGE PLANS:
                     outputColumnNames: key, value, ds, hr
                     Statistics: Num rows: 196 Data size: 257552 Basic stats: PARTIAL Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       keys: ds (type: string), hr (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -452,11 +452,11 @@ STAGE PLANS:
               /spart/ds=2008-04-08/hr=11 [spart]
               /spart/ds=2008-04-08/hr=12 [spart]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -759,7 +759,7 @@ STAGE PLANS:
                     outputColumnNames: key, value, ds, hr
                     Statistics: Num rows: 196 Data size: 257552 Basic stats: PARTIAL Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       keys: ds (type: string), hr (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -859,11 +859,11 @@ STAGE PLANS:
               /spart/ds=2008-04-08/hr=11 [spart]
               /spart/ds=2008-04-08/hr=12 [spart]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -1163,7 +1163,7 @@ STAGE PLANS:
                     outputColumnNames: key, value, ds
                     Statistics: Num rows: 117 Data size: 121232 Basic stats: PARTIAL Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       keys: ds (type: string), '11' (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -1225,11 +1225,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /spart/ds=2008-04-08/hr=11 [spart]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string), '11' (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
diff --git a/ql/src/test/results/clientpositive/llap/column_table_stats_orc.q.out b/ql/src/test/results/clientpositive/llap/column_table_stats_orc.q.out
index ed1f15c..d57c03f 100644
--- a/ql/src/test/results/clientpositive/llap/column_table_stats_orc.q.out
+++ b/ql/src/test/results/clientpositive/llap/column_table_stats_orc.q.out
@@ -89,7 +89,7 @@ STAGE PLANS:
                     outputColumnNames: key, value
                     Statistics: Num rows: 1 Data size: 170 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       minReductionHashAggr: 0.4
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -143,11 +143,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /s_n0 [s_n0]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
@@ -353,7 +353,7 @@ STAGE PLANS:
                     outputColumnNames: key, value, ds, hr
                     Statistics: Num rows: 2 Data size: 1076 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       keys: ds (type: string), hr (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -447,11 +447,11 @@ STAGE PLANS:
               /spart_n0/ds=2008-04-08/hr=11 [spart_n0]
               /spart_n0/ds=2008-04-08/hr=12 [spart_n0]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -749,7 +749,7 @@ STAGE PLANS:
                     outputColumnNames: key, value, ds
                     Statistics: Num rows: 1 Data size: 354 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       keys: ds (type: string), '11' (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -808,11 +808,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /spart_n0/ds=2008-04-08/hr=11 [spart_n0]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string), '11' (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
diff --git a/ql/src/test/results/clientpositive/llap/columnstats_partlvl.q.out b/ql/src/test/results/clientpositive/llap/columnstats_partlvl.q.out
index 1c0f102..99549ea 100644
--- a/ql/src/test/results/clientpositive/llap/columnstats_partlvl.q.out
+++ b/ql/src/test/results/clientpositive/llap/columnstats_partlvl.q.out
@@ -69,7 +69,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid
                     Statistics: Num rows: 3 Data size: 1062 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid)
                       keys: 2000.0D (type: double)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -85,10 +85,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 keys: 2000.0D (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
@@ -154,7 +154,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid
                     Statistics: Num rows: 3 Data size: 1062 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid)
                       keys: 2000.0D (type: double)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -217,11 +217,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /employee_part/employeesalary=2000.0 [employee_part]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 keys: 2000.0D (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
@@ -356,7 +356,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid
                     Statistics: Num rows: 3 Data size: 1062 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid)
                       keys: 4000.0D (type: double)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -372,10 +372,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 keys: 4000.0D (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
@@ -441,7 +441,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid
                     Statistics: Num rows: 3 Data size: 1062 Basic stats: PARTIAL Column stats: NONE
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid)
                       keys: 4000.0D (type: double)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -504,11 +504,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /employee_part/employeesalary=4000.0 [employee_part]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 keys: 4000.0D (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
@@ -605,7 +605,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid, employeename
                     Statistics: Num rows: 13 Data size: 52 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll'), max(length(employeename)), avg(COALESCE(length(employeename),0)), count(employeename), compute_bit_vector(employeename, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid), max(length(employeename)), avg(COALESCE(length(employeename),0)), count(employeename), compute_bit_vector_hll(employeename)
                       keys: 2000.0D (type: double)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -621,10 +621,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: 2000.0D (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -741,7 +741,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid, employeename, employeesalary
                     Statistics: Num rows: 26 Data size: 2596 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll'), max(length(employeename)), avg(COALESCE(length(employeename),0)), count(employeename), compute_bit_vector(employeename, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid), max(length(employeename)), avg(COALESCE(length(employeename),0)), count(employeename), compute_bit_vector_hll(employeename)
                       keys: employeesalary (type: double)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -757,10 +757,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: double)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
@@ -881,7 +881,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid, employeename
                     Statistics: Num rows: 26 Data size: 2300 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll'), max(length(employeename)), avg(COALESCE(length(employeename),0)), count(employeename), compute_bit_vector(employeename, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid), max(length(employeename)), avg(COALESCE(length(employeename),0)), count(employeename), compute_bit_vector_hll(employeename)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -894,10 +894,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: PARTIAL
diff --git a/ql/src/test/results/clientpositive/llap/columnstats_partlvl_dp.q.out b/ql/src/test/results/clientpositive/llap/columnstats_partlvl_dp.q.out
index 5eb9b11..ddb8748 100644
--- a/ql/src/test/results/clientpositive/llap/columnstats_partlvl_dp.q.out
+++ b/ql/src/test/results/clientpositive/llap/columnstats_partlvl_dp.q.out
@@ -105,7 +105,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid, employeename, country
                     Statistics: Num rows: 1 Data size: 1012 Basic stats: PARTIAL Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(employeename)), avg(COALESCE(length(employeename),0)), count(1), count(employeename), compute_bit_vector(employeename, 'hll'), min(employeeid), max(employeeid), count(employeeid), compute_bit_vector(employeeid, 'hll')
+                      aggregations: max(length(employeename)), avg(COALESCE(length(employeename),0)), count(1), count(employeename), compute_bit_vector_hll(employeename), min(employeeid), max(employeeid), count(employeeid), compute_bit_vector_hll(employeeid)
                       keys: 4000.0D (type: double), country (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -121,10 +121,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: 4000.0D (type: double), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
@@ -263,7 +263,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid, country
                     Statistics: Num rows: 3 Data size: 2254 Basic stats: PARTIAL Column stats: PARTIAL
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid)
                       keys: 2000.0D (type: double), country (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -279,10 +279,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 keys: 2000.0D (type: double), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
@@ -419,7 +419,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid, employeesalary, country
                     Statistics: Num rows: 31 Data size: 6072 Basic stats: PARTIAL Column stats: PARTIAL
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid)
                       keys: employeesalary (type: double), country (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -435,10 +435,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 keys: KEY._col0 (type: double), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
@@ -573,7 +573,7 @@ STAGE PLANS:
                     outputColumnNames: employeeid, employeename, employeesalary, country
                     Statistics: Num rows: 54 Data size: 15386 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector(employeeid, 'hll'), max(length(employeename)), avg(COALESCE(length(employeename),0)), count(employeename), compute_bit_vector(employeename, 'hll')
+                      aggregations: min(employeeid), max(employeeid), count(1), count(employeeid), compute_bit_vector_hll(employeeid), max(length(employeename)), avg(COALESCE(length(employeename),0)), count(employeename), compute_bit_vector_hll(employeename)
                       keys: employeesalary (type: double), country (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -589,10 +589,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: double), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
diff --git a/ql/src/test/results/clientpositive/llap/columnstats_quoting.q.out b/ql/src/test/results/clientpositive/llap/columnstats_quoting.q.out
index 695cc29..42385c2 100644
--- a/ql/src/test/results/clientpositive/llap/columnstats_quoting.q.out
+++ b/ql/src/test/results/clientpositive/llap/columnstats_quoting.q.out
@@ -42,7 +42,7 @@ STAGE PLANS:
                     outputColumnNames: user id, user name
                     Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(user id), max(user id), count(1), count(user id), compute_bit_vector(user id, 'hll'), max(length(user name)), avg(COALESCE(length(user name),0)), count(user name), compute_bit_vector(user name, 'hll')
+                      aggregations: min(user id), max(user id), count(1), count(user id), compute_bit_vector_hll(user id), max(length(user name)), avg(COALESCE(length(user name),0)), count(user name), compute_bit_vector_hll(user name)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -55,10 +55,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 600 Basic stats: COMPLETE Column stats: NONE
@@ -124,7 +124,7 @@ STAGE PLANS:
                     outputColumnNames: user id
                     Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(user id), max(user id), count(1), count(user id), compute_bit_vector(user id, 'hll')
+                      aggregations: min(user id), max(user id), count(1), count(user id), compute_bit_vector_hll(user id)
                       minReductionHashAggr: 0.4
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
@@ -137,10 +137,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1 Data size: 176 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/columnstats_tbllvl.q.out b/ql/src/test/results/clientpositive/llap/columnstats_tbllvl.q.out
index 0853b2a..c4c1510 100644
--- a/ql/src/test/results/clientpositive/llap/columnstats_tbllvl.q.out
+++ b/ql/src/test/results/clientpositive/llap/columnstats_tbllvl.q.out
@@ -72,7 +72,7 @@ STAGE PLANS:
                     outputColumnNames: sourceip, adrevenue, avgtimeonsite
                     Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector(sourceip, 'hll'), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector(avgtimeonsite, 'hll'), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector(adrevenue, 'hll')
+                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector_hll(sourceip), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector_hll(avgtimeonsite), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector_hll(adrevenue)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -85,10 +85,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 752 Basic stats: COMPLETE Column stats: NONE
@@ -148,7 +148,7 @@ STAGE PLANS:
                     outputColumnNames: sourceip, adrevenue, avgtimeonsite
                     Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector(sourceip, 'hll'), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector(avgtimeonsite, 'hll'), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector(adrevenue, 'hll')
+                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector_hll(sourceip), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector_hll(avgtimeonsite), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector_hll(adrevenue)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -204,11 +204,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /uservisits_web_text_none [uservisits_web_text_none]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 752 Basic stats: COMPLETE Column stats: NONE
@@ -295,7 +295,7 @@ STAGE PLANS:
                     outputColumnNames: sourceip, desturl, visitdate, adrevenue, useragent, ccode, lcode, skeyword, avgtimeonsite
                     Statistics: Num rows: 55 Data size: 65391 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector(sourceip, 'hll'), max(length(desturl)), avg(COALESCE(length(desturl),0)), count(desturl), compute_bit_vector(desturl, 'hll'), max(length(visitdate)), avg(COALESCE(length(visitdate),0)), count(visitdate), compute_bit_vector(visitdate, 'hll'), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector(adrevenue, 'hll'), max(length(useragent)) [...]
+                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector_hll(sourceip), max(length(desturl)), avg(COALESCE(length(desturl),0)), count(desturl), compute_bit_vector_hll(desturl), max(length(visitdate)), avg(COALESCE(length(visitdate),0)), count(visitdate), compute_bit_vector_hll(visitdate), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector_hll(adrevenue), max(length(useragent)), avg(COALES [...]
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36
@@ -308,10 +308,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16), max(VALUE._col17), avg(VALUE._col18), count(VALUE._col19), compute_bit_vector(V [...]
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16), max(VALUE._col17), avg(VALUE._col18), count(VALUE._col19), comp [...]
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36
                 Statistics: Num rows: 1 Data size: 1476 Basic stats: COMPLETE Column stats: PARTIAL
@@ -456,7 +456,7 @@ STAGE PLANS:
                     outputColumnNames: a, b, c, d, e
                     Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), max(length(c)), avg(COALESCE(length(c),0)), count(c), compute_bit_vector(c, 'hll'), count(CASE WHEN (d is true) THEN (1) ELSE (null) END), count(CASE WHEN (d is false) THEN (1) ELSE (null) END), count(d), max(length(e)), avg(COALESCE(length(e),0)), count(e)
+                      aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(b), max(b), count(b), compute_bit_vector_hll(b), max(length(c)), avg(COALESCE(length(c),0)), count(c), compute_bit_vector_hll(c), count(CASE WHEN (d is true) THEN (1) ELSE (null) END), count(CASE WHEN (d is false) THEN (1) ELSE (null) END), count(d), max(length(e)), avg(COALESCE(length(e),0)), count(e)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -469,10 +469,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), count(VALUE._col13), count(VALUE._col14), count(VALUE._col15), max(VALUE._col16), avg(VALUE._col17), count(VALUE._col18)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), count(VALUE._col13), count(VALUE._col14), count(VALUE._col15), max(VALUE._col16), avg(VALUE._col17), count(VALUE._col18)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
                 Statistics: Num rows: 1 Data size: 1024 Basic stats: COMPLETE Column stats: NONE
@@ -623,7 +623,7 @@ STAGE PLANS:
                     outputColumnNames: sourceip, adrevenue, avgtimeonsite
                     Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector(sourceip, 'hll'), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector(avgtimeonsite, 'hll'), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector(adrevenue, 'hll')
+                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector_hll(sourceip), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector_hll(avgtimeonsite), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector_hll(adrevenue)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -636,10 +636,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 752 Basic stats: COMPLETE Column stats: NONE
@@ -699,7 +699,7 @@ STAGE PLANS:
                     outputColumnNames: sourceip, adrevenue, avgtimeonsite
                     Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector(sourceip, 'hll'), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector(avgtimeonsite, 'hll'), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector(adrevenue, 'hll')
+                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector_hll(sourceip), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector_hll(avgtimeonsite), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector_hll(adrevenue)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -755,11 +755,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /dummydb.db/uservisits_in_dummy_db [uservisits_in_dummy_db]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 752 Basic stats: COMPLETE Column stats: NONE
@@ -846,7 +846,7 @@ STAGE PLANS:
                     outputColumnNames: sourceip, desturl, visitdate, adrevenue, useragent, ccode, lcode, skeyword, avgtimeonsite
                     Statistics: Num rows: 55 Data size: 65391 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector(sourceip, 'hll'), max(length(desturl)), avg(COALESCE(length(desturl),0)), count(desturl), compute_bit_vector(desturl, 'hll'), max(length(visitdate)), avg(COALESCE(length(visitdate),0)), count(visitdate), compute_bit_vector(visitdate, 'hll'), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector(adrevenue, 'hll'), max(length(useragent)) [...]
+                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector_hll(sourceip), max(length(desturl)), avg(COALESCE(length(desturl),0)), count(desturl), compute_bit_vector_hll(desturl), max(length(visitdate)), avg(COALESCE(length(visitdate),0)), count(visitdate), compute_bit_vector_hll(visitdate), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector_hll(adrevenue), max(length(useragent)), avg(COALES [...]
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36
@@ -859,10 +859,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16), max(VALUE._col17), avg(VALUE._col18), count(VALUE._col19), compute_bit_vector(V [...]
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16), max(VALUE._col17), avg(VALUE._col18), count(VALUE._col19), comp [...]
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36
                 Statistics: Num rows: 1 Data size: 1476 Basic stats: COMPLETE Column stats: PARTIAL
diff --git a/ql/src/test/results/clientpositive/llap/compute_bit_vector.q.out b/ql/src/test/results/clientpositive/llap/compute_bit_vector.q.out
new file mode 100644
index 0000000..45d0253
--- /dev/null
+++ b/ql/src/test/results/clientpositive/llap/compute_bit_vector.q.out
@@ -0,0 +1,127 @@
+PREHOOK: query: select hex(compute_bit_vector_hll(ctinyint)) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(ctinyint)) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA48201002400000000000010000000000000000030000000010000300020000020000000001010000000000000101000000000000000000000000000010010000000000000000000000000000000202000000000003000000000005010000000100020000100000004001100200000600000000020000001000000000040000101000000000000030000000000000300000000000000010000001004000000000241000001000000002000200000000020000000000101000000000000000000000010000020000030000000000000000000300000000000000000000000000100000332000000000000000000000000000000 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(csmallint)) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(csmallint)) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA5A22C18C6318C4210C42208421904229CE61886428CA2208421188528C22104653906421024214A519843114632084431823314E5210C32046211C6530C8518C4328C6430C6520C841846520465214A4408C4190840986428C68114823144318CC618C4321067308A530824228C328C6428C6528CC218C6518D6329043208C53886310C8119C65108A310C443104618C43108C22108318CC31144431C812106221C83184672184318C6610C42098851944340C6328842188C21A06210C4319045288A2190A220E8722D0551C82108A3288C418C6420CC518D2330C2448C4220482208A23984220465190A320C22080A511C63 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(cint)) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(cint)) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA4D82F355423757423324242535266445622512252654628243327566331021425332322234444224356644344335634333A52237434632253355232245363346343435355344234562333533433766343641515285338247525475452543775324445556724113632282212255254252435175445305303633622465275141334236513531489265253453320343526333255323365534223525B525245343456143476632236421635533625436354235443343422264422242455F5524325436624235542518842363263435344245433234455315543383344423545553484532532363465333416555735535643466322 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(cast (cbigint as decimal))) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(cast (cbigint as decimal))) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA4FC2F43523724243563425354143726322323343235323616341434423426964326321212223322365252351626433346235765453853644333547438354544533644562234735221443555A563735533653345433358526447263623538136477842612132473393634653344454334132343206223266334145434546317326233261624824364134452425448264053352213342452222644444364222333245292653643326314543427433565224543257435422654332741634323644544A84334433A55532232544238364305376544422344238113423426746738424343614334342342325273691635543653433 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(cfloat)) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(cfloat)) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA48301000000000000000036000000000000001000000121000004133200010020002000040000000000000000000000000020000000000000000000000000000002000000000000000000002200000000000000000000000000000000100000000000010000000000001000000000000050000002000000000020000100000000000010000000300000000100000001000001004000000000000000000000010200001300000100000000100003000000020000000000001000000000000000000000000002000100000200000001100000000000200000030040000010009000000000100000000000010100200020002000 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(cdouble)) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(cdouble)) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA4972B5352374331432543572643434523451325412354314433747333215353433265124362234367423235623675453321331233343223225723724355445434432456458363566364323132744437433445356664521453675606424915622346133238523343222256234311641234363534256122426625A55F61732412584333343583346525745414223233753323323455153352221422424386246192A296215632334514433272445346743533254424344746485322634537233534635333262544479443A633637343343443261332823425124515223534614654354135513666332223744533638226621132 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(cstring1)) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(cstring1)) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA4DB2E3647244A532346626253322543632115463932414344744575552563421142224754323176543623532642443426469524385224623223462365633227373243652242432323367343413724444A4355732254443432541331583444354232272284434516625434446823342864465352323243433322433254414434426643524813385351533543149353222224486663433B445334522353424364334934236126735849345158352133523625542644725445472252545644542662532553443447353462346491362465637454211123A322646265B34242433442543274244253357745624255363535352483 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(cstring2)) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(cstring2)) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA4EA2F65244334311225527575342336234466332472242323465535235565564252754324334526352333355645654875F61935835854522433268583342825233146351543734273343543322542232354346446333446372225421524371443743332C633436732236534582644544544135A33249442266444758523253453655232334434524522625568429224264277822235665433345342543633345523213138513233171474131452413D3444345114552315542541257623123736245643335347553332352234354523134423432444314533362551473333324307255442452B321354365263525256335235 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(ctimestamp1)) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(ctimestamp1)) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA42300000000000000000000000000000000000030000000000000001300000000000100000000000000000000000000000000000000000000000000000000000000000000000000000000000100000000000000000000000000000000000000000000000000100100000000000000000000000000000000000000000000000000000000000000500000000000020000000000000000000000000000000000000000002000000000000000000000000000000000000000000030000000000900000000200700000000010000000003000000000000000000000000000000000000000000000000000000000000000000000000 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(cast (ctimestamp2 as date))) from alltypesorc
+PREHOOK: type: QUERY
+PREHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(cast (ctimestamp2 as date))) from alltypesorc
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@alltypesorc
+#### A masked pattern was here ####
+484C4CA301000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000C0000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000 [...]
+PREHOOK: query: create table test_compute_bit_vector (val1 string, val2 string)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@test_compute_bit_vector
+POSTHOOK: query: create table test_compute_bit_vector (val1 string, val2 string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@test_compute_bit_vector
+Warning: Shuffle Join MERGEJOIN[16][tables = [$hdt$_0, $hdt$_1]] in Stage 'Reducer 2' is a cross product
+PREHOOK: query: insert overwrite table test_compute_bit_vector select a.value, b.value from src as a, src as b
+PREHOOK: type: QUERY
+PREHOOK: Input: default@src
+PREHOOK: Output: default@test_compute_bit_vector
+POSTHOOK: query: insert overwrite table test_compute_bit_vector select a.value, b.value from src as a, src as b
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@src
+POSTHOOK: Output: default@test_compute_bit_vector
+POSTHOOK: Lineage: test_compute_bit_vector.val1 SIMPLE [(src)a.FieldSchema(name:value, type:string, comment:default), ]
+POSTHOOK: Lineage: test_compute_bit_vector.val2 SIMPLE [(src)b.FieldSchema(name:value, type:string, comment:default), ]
+PREHOOK: query: select hex(compute_bit_vector_hll(val1)) from test_compute_bit_vector
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_compute_bit_vector
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(val1)) from test_compute_bit_vector
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test_compute_bit_vector
+#### A masked pattern was here ####
+484C4CA4B302000000000000202000000020001011000000000000000001430120012000000000020000000101000000000100363000010000001001300050001104000020000000011000000100001132010010100030000000010000200001200000020000011030004010311000000000010602100020000001000000020000003100000300000010000000000002010000010002000000000201604011001000030002001000000000000206011000243000001000040100001310001000101002030110000310100020010000000306020010000040003012024000000000321110104100040005003001020161000000046000 [...]
+PREHOOK: query: select hex(compute_bit_vector_hll(val2)) from test_compute_bit_vector
+PREHOOK: type: QUERY
+PREHOOK: Input: default@test_compute_bit_vector
+#### A masked pattern was here ####
+POSTHOOK: query: select hex(compute_bit_vector_hll(val2)) from test_compute_bit_vector
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@test_compute_bit_vector
+#### A masked pattern was here ####
+484C4CA4B302000000000000202000000020001011000000000000000001430120012000000000020000000101000000000100363000010000001001300050001104000020000000011000000100001132010010100030000000010000200001200000020000011030004010311000000000010602100020000001000000020000003100000300000010000000000002010000010002000000000201604011001000030002001000000000000206011000243000001000040100001310001000101002030110000310100020010000000306020010000040003012024000000000321110104100040005003001020161000000046000 [...]
diff --git a/ql/src/test/results/clientpositive/llap/compute_stats_date.q.out b/ql/src/test/results/clientpositive/llap/compute_stats_date.q.out
index 36c910b..f28c21c 100644
--- a/ql/src/test/results/clientpositive/llap/compute_stats_date.q.out
+++ b/ql/src/test/results/clientpositive/llap/compute_stats_date.q.out
@@ -78,7 +78,7 @@ STAGE PLANS:
                     outputColumnNames: fl_date
                     Statistics: Num rows: 1 Data size: 56 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(fl_date), max(fl_date), count(1), count(fl_date), compute_bit_vector(fl_date, 'hll')
+                      aggregations: min(fl_date), max(fl_date), count(1), count(fl_date), compute_bit_vector_hll(fl_date)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
@@ -91,10 +91,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/constant_prop_2.q.out b/ql/src/test/results/clientpositive/llap/constant_prop_2.q.out
index a56c9f6..d9333ba 100644
--- a/ql/src/test/results/clientpositive/llap/constant_prop_2.q.out
+++ b/ql/src/test/results/clientpositive/llap/constant_prop_2.q.out
@@ -64,7 +64,7 @@ STAGE PLANS:
                     outputColumnNames: key, value
                     Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: PARTIAL
                     Group By Operator
-                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                      aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                       keys: '2008-04-08' (type: string), '11' (type: string)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -80,10 +80,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: '2008-04-08' (type: string), '11' (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
diff --git a/ql/src/test/results/clientpositive/llap/constprog_dp.q.out b/ql/src/test/results/clientpositive/llap/constprog_dp.q.out
index 96002cc..b9810a3 100644
--- a/ql/src/test/results/clientpositive/llap/constprog_dp.q.out
+++ b/ql/src/test/results/clientpositive/llap/constprog_dp.q.out
@@ -59,7 +59,7 @@ STAGE PLANS:
                       outputColumnNames: key, value, ds
                       Statistics: Num rows: 1000 Data size: 272000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                        aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                         keys: ds (type: string)
                         minReductionHashAggr: 0.99
                         mode: hash
@@ -72,13 +72,13 @@ STAGE PLANS:
                           Map-reduce partition columns: _col0 (type: string)
                           Statistics: Num rows: 1 Data size: 566 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col1 (type: int), _col2 (type: struct<count:bigint,sum:double,input:int>), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: struct<count:bigint,sum:double,input:int>), _col8 (type: bigint), _col9 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/constprog_type.q.out b/ql/src/test/results/clientpositive/llap/constprog_type.q.out
index 0dbbbc9..185dbe6 100644
--- a/ql/src/test/results/clientpositive/llap/constprog_type.q.out
+++ b/ql/src/test/results/clientpositive/llap/constprog_type.q.out
@@ -57,7 +57,7 @@ STAGE PLANS:
                       outputColumnNames: d, t
                       Statistics: Num rows: 500 Data size: 48000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: min(d), max(d), count(1), count(d), compute_bit_vector(d, 'hll'), min(t), max(t), count(t), compute_bit_vector(t, 'hll')
+                        aggregations: min(d), max(d), count(1), count(d), compute_bit_vector_hll(d), min(t), max(t), count(t), compute_bit_vector_hll(t)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -67,13 +67,13 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: date), _col1 (type: date), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: timestamp), _col6 (type: timestamp), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/cp_sel.q.out b/ql/src/test/results/clientpositive/llap/cp_sel.q.out
index cd2c967..5dcae92 100644
--- a/ql/src/test/results/clientpositive/llap/cp_sel.q.out
+++ b/ql/src/test/results/clientpositive/llap/cp_sel.q.out
@@ -146,7 +146,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), VALUE._col0 (type: string), VALUE._col1 (type: string), VALUE._col2 (type: string)
@@ -165,7 +165,7 @@ STAGE PLANS:
                   outputColumnNames: key, value, ds, hr
                   Statistics: Num rows: 1000 Data size: 356000 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: max(length(key)), avg(COALESCE(length(key),0)), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     keys: ds (type: string), hr (type: string)
                     minReductionHashAggr: 0.99
                     mode: hash
@@ -179,10 +179,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 650 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col2 (type: int), _col3 (type: struct<count:bigint,sum:double,input:int>), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: struct<count:bigint,sum:double,input:int>), _col9 (type: bigint), _col10 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string), KEY._col1 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10
diff --git a/ql/src/test/results/clientpositive/llap/ctas.q.out b/ql/src/test/results/clientpositive/llap/ctas.q.out
index 0a66f75..a0ad8b7 100644
--- a/ql/src/test/results/clientpositive/llap/ctas.q.out
+++ b/ql/src/test/results/clientpositive/llap/ctas.q.out
@@ -85,7 +85,7 @@ STAGE PLANS:
                       sort order: ++
                       Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 10
@@ -107,7 +107,7 @@ STAGE PLANS:
                     outputColumnNames: col1, col2
                     Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                       minReductionHashAggr: 0.9
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -118,10 +118,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
@@ -298,7 +298,7 @@ STAGE PLANS:
                       sort order: ++
                       Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 10
@@ -320,7 +320,7 @@ STAGE PLANS:
                     outputColumnNames: col1, col2
                     Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                       minReductionHashAggr: 0.9
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -331,10 +331,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
@@ -511,7 +511,7 @@ STAGE PLANS:
                       sort order: ++
                       Statistics: Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 10
@@ -533,7 +533,7 @@ STAGE PLANS:
                     outputColumnNames: col1, col2
                     Statistics: Num rows: 10 Data size: 1920 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(col1), max(col1), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                      aggregations: min(col1), max(col1), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                       minReductionHashAggr: 0.9
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -544,10 +544,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 408 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: double), _col1 (type: double), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 340 Basic stats: COMPLETE Column stats: COMPLETE
@@ -788,7 +788,7 @@ STAGE PLANS:
                       sort order: ++
                       Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 10
@@ -810,7 +810,7 @@ STAGE PLANS:
                     outputColumnNames: col1, col2
                     Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                       minReductionHashAggr: 0.9
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -821,10 +821,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1003,7 +1003,7 @@ STAGE PLANS:
                       sort order: ++
                       Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 10
@@ -1025,7 +1025,7 @@ STAGE PLANS:
                     outputColumnNames: col1, col2
                     Statistics: Num rows: 10 Data size: 1780 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                       minReductionHashAggr: 0.9
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -1036,10 +1036,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/ctas_colname.q.out b/ql/src/test/results/clientpositive/llap/ctas_colname.q.out
index be8c66b..2466ade 100644
--- a/ql/src/test/results/clientpositive/llap/ctas_colname.q.out
+++ b/ql/src/test/results/clientpositive/llap/ctas_colname.q.out
@@ -47,7 +47,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 20
@@ -69,7 +69,7 @@ STAGE PLANS:
                     outputColumnNames: col1, col2, col3, col4
                     Statistics: Num rows: 20 Data size: 7400 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll'), min(col3), max(col3), count(col3), compute_bit_vector(col3, 'hll'), max(length(col4)), avg(COALESCE(length(col4),0)), count(col4), compute_bit_vector(col4, 'hll')
+                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2), min(col3), max(col3), count(col3), compute_bit_vector_hll(col3), max(length(col4)), avg(COALESCE(length(col4),0)), count(col4), compute_bit_vector_hll(col4)
                       minReductionHashAggr: 0.95
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
@@ -80,10 +80,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 872 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: double), _col10 (type: double), _col11 (type: bigint), _col12 (type: binary), _col13 (type: int), _col14 (type: struct<count:bigint,sum:double,input:int>), _col15 (type: bigi [...]
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16
                 Statistics: Num rows: 1 Data size: 668 Basic stats: COMPLETE Column stats: COMPLETE
@@ -245,7 +245,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: string), KEY.reducesinkkey1 (type: string)
@@ -289,7 +289,7 @@ STAGE PLANS:
                       outputColumnNames: col1, col2, col3
                       Statistics: Num rows: 25 Data size: 4475 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll'), min(col3), max(col3), count(col3), compute_bit_vector(col3, 'hll')
+                        aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2), min(col3), max(col3), count(col3), compute_bit_vector_hll(col3)
                         minReductionHashAggr: 0.96
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -300,10 +300,10 @@ STAGE PLANS:
                           Statistics: Num rows: 1 Data size: 632 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: int), _col11 (type: bigint), _col12 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 496 Basic stats: COMPLETE Column stats: COMPLETE
@@ -510,7 +510,7 @@ STAGE PLANS:
                         TopN Hash Memory Usage: 0.1
                         value expressions: _col0 (type: string), _col1 (type: string), _col2 (type: string)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 20
@@ -532,7 +532,7 @@ STAGE PLANS:
                     outputColumnNames: col1, col2, col3
                     Statistics: Num rows: 20 Data size: 7240 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll'), max(length(col3)), avg(COALESCE(length(col3),0)), count(col3), compute_bit_vector(col3, 'hll')
+                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2), max(length(col3)), avg(COALESCE(length(col3),0)), count(col3), compute_bit_vector_hll(col3)
                       minReductionHashAggr: 0.95
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -543,10 +543,10 @@ STAGE PLANS:
                         Statistics: Num rows: 1 Data size: 704 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: int), _col10 (type: struct<count:bigint,sum:double,input:int>), _col11 (type: bigint), _col12 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 500 Basic stats: COMPLETE Column stats: COMPLETE
@@ -713,7 +713,7 @@ STAGE PLANS:
                       outputColumnNames: col1, col2, col3
                       Statistics: Num rows: 25 Data size: 4575 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll'), min(col3), max(col3), count(col3), compute_bit_vector(col3, 'hll')
+                        aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2), min(col3), max(col3), count(col3), compute_bit_vector_hll(col3)
                         minReductionHashAggr: 0.96
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -723,13 +723,13 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 640 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: double), _col10 (type: double), _col11 (type: bigint), _col12 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE
@@ -906,7 +906,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0)
@@ -927,7 +927,7 @@ STAGE PLANS:
                   outputColumnNames: col1, col2, col3
                   Statistics: Num rows: 250 Data size: 46500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll'), min(col3), max(col3), count(col3), compute_bit_vector(col3, 'hll')
+                    aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2), min(col3), max(col3), count(col3), compute_bit_vector_hll(col3)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -938,10 +938,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 640 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: bigint), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1402,7 +1402,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: count(VALUE._col0)
@@ -1423,7 +1423,7 @@ STAGE PLANS:
                   outputColumnNames: col1, col2, col3
                   Statistics: Num rows: 83 Data size: 15438 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll'), min(col3), max(col3), count(col3), compute_bit_vector(col3, 'hll')
+                    aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2), min(col3), max(col3), count(col3), compute_bit_vector_hll(col3)
                     minReductionHashAggr: 0.9879518
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -1434,10 +1434,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 640 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary), _col9 (type: bigint), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 504 Basic stats: COMPLETE Column stats: COMPLETE
@@ -1594,7 +1594,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 aggregations: max(VALUE._col0)
@@ -1622,7 +1622,7 @@ STAGE PLANS:
                       outputColumnNames: col1, col2
                       Statistics: Num rows: 83 Data size: 22493 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                        aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                         minReductionHashAggr: 0.9879518
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -1633,10 +1633,10 @@ STAGE PLANS:
                           Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/ctas_uses_database_location.q.out b/ql/src/test/results/clientpositive/llap/ctas_uses_database_location.q.out
index bc8f5f7..7684c82 100644
--- a/ql/src/test/results/clientpositive/llap/ctas_uses_database_location.q.out
+++ b/ql/src/test/results/clientpositive/llap/ctas_uses_database_location.q.out
@@ -61,7 +61,7 @@ STAGE PLANS:
                       outputColumnNames: col1, col2
                       Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                        aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                         minReductionHashAggr: 0.99
                         mode: hash
                         outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -71,13 +71,13 @@ STAGE PLANS:
                           sort order: 
                           Statistics: Num rows: 1 Data size: 472 Basic stats: COMPLETE Column stats: COMPLETE
                           value expressions: _col0 (type: int), _col1 (type: struct<count:bigint,sum:double,input:int>), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: struct<count:bigint,sum:double,input:int>), _col7 (type: bigint), _col8 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/cte_mat_7.q.out b/ql/src/test/results/clientpositive/llap/cte_mat_7.q.out
index 05ae275..6054868 100644
--- a/ql/src/test/results/clientpositive/llap/cte_mat_7.q.out
+++ b/ql/src/test/results/clientpositive/llap/cte_mat_7.q.out
@@ -109,7 +109,7 @@ STAGE PLANS:
                   Statistics: Num rows: 7 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
                   value expressions: _col1 (type: bigint)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: bigint)
@@ -128,7 +128,7 @@ STAGE PLANS:
                   outputColumnNames: col1, col2
                   Statistics: Num rows: 7 Data size: 84 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(col1), max(col1), count(1), count(col1), compute_bit_vector(col1, 'hll'), min(col2), max(col2), count(col2), compute_bit_vector(col2, 'hll')
+                    aggregations: min(col1), max(col1), count(1), count(col1), compute_bit_vector_hll(col1), min(col2), max(col2), count(col2), compute_bit_vector_hll(col2)
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                     Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/disable_merge_for_bucketing.q.out b/ql/src/test/results/clientpositive/llap/disable_merge_for_bucketing.q.out
index e8b1250..bdddf05 100644
--- a/ql/src/test/results/clientpositive/llap/disable_merge_for_bucketing.q.out
+++ b/ql/src/test/results/clientpositive/llap/disable_merge_for_bucketing.q.out
@@ -95,7 +95,7 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /src [src]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Select Operator
@@ -135,7 +135,7 @@ STAGE PLANS:
                   outputColumnNames: key, value
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector(key, 'hll'), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector(value, 'hll')
+                    aggregations: min(key), max(key), count(1), count(key), compute_bit_vector_hll(key), max(length(value)), avg(COALESCE(length(value),0)), count(value), compute_bit_vector_hll(value)
                     mode: complete
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                     Statistics: Num rows: 1 Data size: 332 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/display_colstats_tbllvl.q.out b/ql/src/test/results/clientpositive/llap/display_colstats_tbllvl.q.out
index 453c346..2b7a3d3 100644
--- a/ql/src/test/results/clientpositive/llap/display_colstats_tbllvl.q.out
+++ b/ql/src/test/results/clientpositive/llap/display_colstats_tbllvl.q.out
@@ -97,7 +97,7 @@ STAGE PLANS:
                     outputColumnNames: sourceip, adrevenue, avgtimeonsite
                     Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector(sourceip, 'hll'), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector(avgtimeonsite, 'hll'), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector(adrevenue, 'hll')
+                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector_hll(sourceip), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector_hll(avgtimeonsite), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector_hll(adrevenue)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -110,10 +110,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 752 Basic stats: COMPLETE Column stats: NONE
@@ -173,7 +173,7 @@ STAGE PLANS:
                     outputColumnNames: sourceip, adrevenue, avgtimeonsite
                     Statistics: Num rows: 1 Data size: 192 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector(sourceip, 'hll'), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector(avgtimeonsite, 'hll'), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector(adrevenue, 'hll')
+                      aggregations: max(length(sourceip)), avg(COALESCE(length(sourceip),0)), count(1), count(sourceip), compute_bit_vector_hll(sourceip), min(avgtimeonsite), max(avgtimeonsite), count(avgtimeonsite), compute_bit_vector_hll(avgtimeonsite), min(adrevenue), max(adrevenue), count(adrevenue), compute_bit_vector_hll(adrevenue)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
@@ -229,11 +229,11 @@ STAGE PLANS:
             Truncated Path -> Alias:
               /uservisits_web_text_none_n0 [uservisits_web_text_none_n0]
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Needs Tagging: false
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12
                 Statistics: Num rows: 1 Data size: 752 Basic stats: COMPLETE Column stats: NONE
@@ -416,7 +416,7 @@ STAGE PLANS:
                     outputColumnNames: a, b, c, d, e
                     Statistics: Num rows: 1 Data size: 344 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), max(length(c)), avg(COALESCE(length(c),0)), count(c), compute_bit_vector(c, 'hll'), count(CASE WHEN (d is true) THEN (1) ELSE (null) END), count(CASE WHEN (d is false) THEN (1) ELSE (null) END), count(d), max(length(e)), avg(COALESCE(length(e),0)), count(e)
+                      aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(b), max(b), count(b), compute_bit_vector_hll(b), max(length(c)), avg(COALESCE(length(c),0)), count(c), compute_bit_vector_hll(c), count(CASE WHEN (d is true) THEN (1) ELSE (null) END), count(CASE WHEN (d is false) THEN (1) ELSE (null) END), count(d), max(length(e)), avg(COALESCE(length(e),0)), count(e)
                       minReductionHashAggr: 0.99
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -429,10 +429,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), count(VALUE._col13), count(VALUE._col14), count(VALUE._col15), max(VALUE._col16), avg(VALUE._col17), count(VALUE._col18)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), count(VALUE._col13), count(VALUE._col14), count(VALUE._col15), max(VALUE._col16), avg(VALUE._col17), count(VALUE._col18)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
                 Statistics: Num rows: 1 Data size: 1024 Basic stats: COMPLETE Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out b/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out
index bcea4ae..9256311 100644
--- a/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out
+++ b/ql/src/test/results/clientpositive/llap/dp_counter_mm.q.out
@@ -24,16 +24,16 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2_n5: 84
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
-   RECORDS_OUT_OPERATOR_FIL_12: 84
-   RECORDS_OUT_OPERATOR_FS_11: 57
-   RECORDS_OUT_OPERATOR_FS_4: 84
-   RECORDS_OUT_OPERATOR_GBY_7: 57
-   RECORDS_OUT_OPERATOR_GBY_9: 57
+   RECORDS_OUT_OPERATOR_FIL_13: 84
+   RECORDS_OUT_OPERATOR_FS_15: 84
+   RECORDS_OUT_OPERATOR_FS_21: 57
+   RECORDS_OUT_OPERATOR_GBY_17: 57
+   RECORDS_OUT_OPERATOR_GBY_19: 57
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_8: 57
-   RECORDS_OUT_OPERATOR_SEL_10: 57
-   RECORDS_OUT_OPERATOR_SEL_2: 84
-   RECORDS_OUT_OPERATOR_SEL_6: 84
+   RECORDS_OUT_OPERATOR_RS_18: 57
+   RECORDS_OUT_OPERATOR_SEL_14: 84
+   RECORDS_OUT_OPERATOR_SEL_16: 84
+   RECORDS_OUT_OPERATOR_SEL_20: 57
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 84
 Stage-1 INPUT COUNTERS:
@@ -55,16 +55,16 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2_n5: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
-   RECORDS_OUT_OPERATOR_FIL_12: 189
-   RECORDS_OUT_OPERATOR_FS_11: 121
-   RECORDS_OUT_OPERATOR_FS_4: 189
-   RECORDS_OUT_OPERATOR_GBY_7: 121
-   RECORDS_OUT_OPERATOR_GBY_9: 121
+   RECORDS_OUT_OPERATOR_FIL_13: 189
+   RECORDS_OUT_OPERATOR_FS_15: 189
+   RECORDS_OUT_OPERATOR_FS_21: 121
+   RECORDS_OUT_OPERATOR_GBY_17: 121
+   RECORDS_OUT_OPERATOR_GBY_19: 121
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_8: 121
-   RECORDS_OUT_OPERATOR_SEL_10: 121
-   RECORDS_OUT_OPERATOR_SEL_2: 189
-   RECORDS_OUT_OPERATOR_SEL_6: 189
+   RECORDS_OUT_OPERATOR_RS_18: 121
+   RECORDS_OUT_OPERATOR_SEL_14: 189
+   RECORDS_OUT_OPERATOR_SEL_16: 189
+   RECORDS_OUT_OPERATOR_SEL_20: 121
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 189
 Stage-1 INPUT COUNTERS:
@@ -94,16 +94,16 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2_n5: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
-   RECORDS_OUT_OPERATOR_FIL_12: 189
-   RECORDS_OUT_OPERATOR_FS_11: 121
-   RECORDS_OUT_OPERATOR_FS_4: 189
-   RECORDS_OUT_OPERATOR_GBY_7: 121
-   RECORDS_OUT_OPERATOR_GBY_9: 121
+   RECORDS_OUT_OPERATOR_FIL_13: 189
+   RECORDS_OUT_OPERATOR_FS_15: 189
+   RECORDS_OUT_OPERATOR_FS_21: 121
+   RECORDS_OUT_OPERATOR_GBY_17: 121
+   RECORDS_OUT_OPERATOR_GBY_19: 121
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_8: 121
-   RECORDS_OUT_OPERATOR_SEL_10: 121
-   RECORDS_OUT_OPERATOR_SEL_2: 189
-   RECORDS_OUT_OPERATOR_SEL_6: 189
+   RECORDS_OUT_OPERATOR_RS_18: 121
+   RECORDS_OUT_OPERATOR_SEL_14: 189
+   RECORDS_OUT_OPERATOR_SEL_16: 189
+   RECORDS_OUT_OPERATOR_SEL_20: 121
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 189
 Stage-1 INPUT COUNTERS:
@@ -125,16 +125,16 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2_n5: 292
    RECORDS_OUT_INTERMEDIATE_Map_1: 184
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
-   RECORDS_OUT_OPERATOR_FIL_12: 292
-   RECORDS_OUT_OPERATOR_FS_11: 184
-   RECORDS_OUT_OPERATOR_FS_4: 292
-   RECORDS_OUT_OPERATOR_GBY_7: 184
-   RECORDS_OUT_OPERATOR_GBY_9: 184
+   RECORDS_OUT_OPERATOR_FIL_13: 292
+   RECORDS_OUT_OPERATOR_FS_15: 292
+   RECORDS_OUT_OPERATOR_FS_21: 184
+   RECORDS_OUT_OPERATOR_GBY_17: 184
+   RECORDS_OUT_OPERATOR_GBY_19: 184
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_8: 184
-   RECORDS_OUT_OPERATOR_SEL_10: 184
-   RECORDS_OUT_OPERATOR_SEL_2: 292
-   RECORDS_OUT_OPERATOR_SEL_6: 292
+   RECORDS_OUT_OPERATOR_RS_18: 184
+   RECORDS_OUT_OPERATOR_SEL_14: 292
+   RECORDS_OUT_OPERATOR_SEL_16: 292
+   RECORDS_OUT_OPERATOR_SEL_20: 184
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 292
 Stage-1 INPUT COUNTERS:
@@ -175,25 +175,25 @@ Stage-2 HIVE COUNTERS:
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
    RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
-   RECORDS_OUT_OPERATOR_FIL_23: 84
-   RECORDS_OUT_OPERATOR_FIL_24: 105
-   RECORDS_OUT_OPERATOR_FS_11: 57
-   RECORDS_OUT_OPERATOR_FS_15: 105
-   RECORDS_OUT_OPERATOR_FS_22: 64
-   RECORDS_OUT_OPERATOR_FS_4: 84
-   RECORDS_OUT_OPERATOR_GBY_18: 64
-   RECORDS_OUT_OPERATOR_GBY_20: 64
-   RECORDS_OUT_OPERATOR_GBY_7: 57
-   RECORDS_OUT_OPERATOR_GBY_9: 57
+   RECORDS_OUT_OPERATOR_FIL_25: 84
+   RECORDS_OUT_OPERATOR_FIL_26: 105
+   RECORDS_OUT_OPERATOR_FS_29: 84
+   RECORDS_OUT_OPERATOR_FS_31: 105
+   RECORDS_OUT_OPERATOR_FS_39: 57
+   RECORDS_OUT_OPERATOR_FS_42: 64
+   RECORDS_OUT_OPERATOR_GBY_33: 57
+   RECORDS_OUT_OPERATOR_GBY_34: 64
+   RECORDS_OUT_OPERATOR_GBY_37: 57
+   RECORDS_OUT_OPERATOR_GBY_40: 64
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_19: 64
-   RECORDS_OUT_OPERATOR_RS_8: 57
-   RECORDS_OUT_OPERATOR_SEL_10: 57
-   RECORDS_OUT_OPERATOR_SEL_13: 105
-   RECORDS_OUT_OPERATOR_SEL_17: 105
-   RECORDS_OUT_OPERATOR_SEL_2: 84
-   RECORDS_OUT_OPERATOR_SEL_21: 64
-   RECORDS_OUT_OPERATOR_SEL_6: 84
+   RECORDS_OUT_OPERATOR_RS_35: 57
+   RECORDS_OUT_OPERATOR_RS_36: 64
+   RECORDS_OUT_OPERATOR_SEL_27: 84
+   RECORDS_OUT_OPERATOR_SEL_28: 105
+   RECORDS_OUT_OPERATOR_SEL_30: 84
+   RECORDS_OUT_OPERATOR_SEL_32: 105
+   RECORDS_OUT_OPERATOR_SEL_38: 57
+   RECORDS_OUT_OPERATOR_SEL_41: 64
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 189
 Stage-2 INPUT COUNTERS:
@@ -220,25 +220,25 @@ Stage-2 HIVE COUNTERS:
    RECORDS_OUT_INTERMEDIATE_Map_1: 184
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
    RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
-   RECORDS_OUT_OPERATOR_FIL_23: 84
-   RECORDS_OUT_OPERATOR_FIL_24: 208
-   RECORDS_OUT_OPERATOR_FS_11: 57
-   RECORDS_OUT_OPERATOR_FS_15: 208
-   RECORDS_OUT_OPERATOR_FS_22: 127
-   RECORDS_OUT_OPERATOR_FS_4: 84
-   RECORDS_OUT_OPERATOR_GBY_18: 127
-   RECORDS_OUT_OPERATOR_GBY_20: 127
-   RECORDS_OUT_OPERATOR_GBY_7: 57
-   RECORDS_OUT_OPERATOR_GBY_9: 57
+   RECORDS_OUT_OPERATOR_FIL_25: 84
+   RECORDS_OUT_OPERATOR_FIL_26: 208
+   RECORDS_OUT_OPERATOR_FS_29: 84
+   RECORDS_OUT_OPERATOR_FS_31: 208
+   RECORDS_OUT_OPERATOR_FS_39: 57
+   RECORDS_OUT_OPERATOR_FS_42: 127
+   RECORDS_OUT_OPERATOR_GBY_33: 57
+   RECORDS_OUT_OPERATOR_GBY_34: 127
+   RECORDS_OUT_OPERATOR_GBY_37: 57
+   RECORDS_OUT_OPERATOR_GBY_40: 127
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_19: 127
-   RECORDS_OUT_OPERATOR_RS_8: 57
-   RECORDS_OUT_OPERATOR_SEL_10: 57
-   RECORDS_OUT_OPERATOR_SEL_13: 208
-   RECORDS_OUT_OPERATOR_SEL_17: 208
-   RECORDS_OUT_OPERATOR_SEL_2: 84
-   RECORDS_OUT_OPERATOR_SEL_21: 127
-   RECORDS_OUT_OPERATOR_SEL_6: 84
+   RECORDS_OUT_OPERATOR_RS_35: 57
+   RECORDS_OUT_OPERATOR_RS_36: 127
+   RECORDS_OUT_OPERATOR_SEL_27: 84
+   RECORDS_OUT_OPERATOR_SEL_28: 208
+   RECORDS_OUT_OPERATOR_SEL_30: 84
+   RECORDS_OUT_OPERATOR_SEL_32: 208
+   RECORDS_OUT_OPERATOR_SEL_38: 57
+   RECORDS_OUT_OPERATOR_SEL_41: 127
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 292
 Stage-2 INPUT COUNTERS:
@@ -274,24 +274,24 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Map_4: 64
    RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
-   RECORDS_OUT_OPERATOR_FIL_20: 84
-   RECORDS_OUT_OPERATOR_FIL_29: 105
-   RECORDS_OUT_OPERATOR_FS_16: 121
-   RECORDS_OUT_OPERATOR_FS_24: 84
-   RECORDS_OUT_OPERATOR_FS_33: 105
-   RECORDS_OUT_OPERATOR_GBY_14: 121
-   RECORDS_OUT_OPERATOR_GBY_26: 57
-   RECORDS_OUT_OPERATOR_GBY_35: 64
+   RECORDS_OUT_OPERATOR_FIL_37: 84
+   RECORDS_OUT_OPERATOR_FIL_47: 105
+   RECORDS_OUT_OPERATOR_FS_40: 84
+   RECORDS_OUT_OPERATOR_FS_46: 121
+   RECORDS_OUT_OPERATOR_FS_50: 105
+   RECORDS_OUT_OPERATOR_GBY_42: 57
+   RECORDS_OUT_OPERATOR_GBY_44: 121
+   RECORDS_OUT_OPERATOR_GBY_52: 64
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_27: 57
-   RECORDS_OUT_OPERATOR_RS_36: 64
-   RECORDS_OUT_OPERATOR_SEL_15: 121
-   RECORDS_OUT_OPERATOR_SEL_21: 84
-   RECORDS_OUT_OPERATOR_SEL_23: 84
-   RECORDS_OUT_OPERATOR_SEL_25: 84
-   RECORDS_OUT_OPERATOR_SEL_30: 105
-   RECORDS_OUT_OPERATOR_SEL_32: 105
-   RECORDS_OUT_OPERATOR_SEL_34: 105
+   RECORDS_OUT_OPERATOR_RS_43: 57
+   RECORDS_OUT_OPERATOR_RS_53: 64
+   RECORDS_OUT_OPERATOR_SEL_38: 84
+   RECORDS_OUT_OPERATOR_SEL_39: 84
+   RECORDS_OUT_OPERATOR_SEL_41: 84
+   RECORDS_OUT_OPERATOR_SEL_45: 121
+   RECORDS_OUT_OPERATOR_SEL_48: 105
+   RECORDS_OUT_OPERATOR_SEL_49: 105
+   RECORDS_OUT_OPERATOR_SEL_51: 105
    RECORDS_OUT_OPERATOR_TS_19: 500
    RECORDS_OUT_OPERATOR_TS_28: 500
    TOTAL_TABLE_ROWS_WRITTEN: 189
@@ -324,24 +324,24 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Map_4: 127
    RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
-   RECORDS_OUT_OPERATOR_FIL_20: 84
-   RECORDS_OUT_OPERATOR_FIL_29: 208
-   RECORDS_OUT_OPERATOR_FS_16: 184
-   RECORDS_OUT_OPERATOR_FS_24: 84
-   RECORDS_OUT_OPERATOR_FS_33: 208
-   RECORDS_OUT_OPERATOR_GBY_14: 184
-   RECORDS_OUT_OPERATOR_GBY_26: 57
-   RECORDS_OUT_OPERATOR_GBY_35: 127
+   RECORDS_OUT_OPERATOR_FIL_37: 84
+   RECORDS_OUT_OPERATOR_FIL_47: 208
+   RECORDS_OUT_OPERATOR_FS_40: 84
+   RECORDS_OUT_OPERATOR_FS_46: 184
+   RECORDS_OUT_OPERATOR_FS_50: 208
+   RECORDS_OUT_OPERATOR_GBY_42: 57
+   RECORDS_OUT_OPERATOR_GBY_44: 184
+   RECORDS_OUT_OPERATOR_GBY_52: 127
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_27: 57
-   RECORDS_OUT_OPERATOR_RS_36: 127
-   RECORDS_OUT_OPERATOR_SEL_15: 184
-   RECORDS_OUT_OPERATOR_SEL_21: 84
-   RECORDS_OUT_OPERATOR_SEL_23: 84
-   RECORDS_OUT_OPERATOR_SEL_25: 84
-   RECORDS_OUT_OPERATOR_SEL_30: 208
-   RECORDS_OUT_OPERATOR_SEL_32: 208
-   RECORDS_OUT_OPERATOR_SEL_34: 208
+   RECORDS_OUT_OPERATOR_RS_43: 57
+   RECORDS_OUT_OPERATOR_RS_53: 127
+   RECORDS_OUT_OPERATOR_SEL_38: 84
+   RECORDS_OUT_OPERATOR_SEL_39: 84
+   RECORDS_OUT_OPERATOR_SEL_41: 84
+   RECORDS_OUT_OPERATOR_SEL_45: 184
+   RECORDS_OUT_OPERATOR_SEL_48: 208
+   RECORDS_OUT_OPERATOR_SEL_49: 208
+   RECORDS_OUT_OPERATOR_SEL_51: 208
    RECORDS_OUT_OPERATOR_TS_19: 500
    RECORDS_OUT_OPERATOR_TS_28: 500
    TOTAL_TABLE_ROWS_WRITTEN: 292
diff --git a/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out b/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out
index 93b4a54..678a421 100644
--- a/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out
+++ b/ql/src/test/results/clientpositive/llap/dp_counter_non_mm.q.out
@@ -24,16 +24,16 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2_n3: 84
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
-   RECORDS_OUT_OPERATOR_FIL_12: 84
-   RECORDS_OUT_OPERATOR_FS_11: 57
-   RECORDS_OUT_OPERATOR_FS_4: 84
-   RECORDS_OUT_OPERATOR_GBY_7: 57
-   RECORDS_OUT_OPERATOR_GBY_9: 57
+   RECORDS_OUT_OPERATOR_FIL_13: 84
+   RECORDS_OUT_OPERATOR_FS_15: 84
+   RECORDS_OUT_OPERATOR_FS_21: 57
+   RECORDS_OUT_OPERATOR_GBY_17: 57
+   RECORDS_OUT_OPERATOR_GBY_19: 57
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_8: 57
-   RECORDS_OUT_OPERATOR_SEL_10: 57
-   RECORDS_OUT_OPERATOR_SEL_2: 84
-   RECORDS_OUT_OPERATOR_SEL_6: 84
+   RECORDS_OUT_OPERATOR_RS_18: 57
+   RECORDS_OUT_OPERATOR_SEL_14: 84
+   RECORDS_OUT_OPERATOR_SEL_16: 84
+   RECORDS_OUT_OPERATOR_SEL_20: 57
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 84
 Stage-1 INPUT COUNTERS:
@@ -55,16 +55,16 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2_n3: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
-   RECORDS_OUT_OPERATOR_FIL_12: 189
-   RECORDS_OUT_OPERATOR_FS_11: 121
-   RECORDS_OUT_OPERATOR_FS_4: 189
-   RECORDS_OUT_OPERATOR_GBY_7: 121
-   RECORDS_OUT_OPERATOR_GBY_9: 121
+   RECORDS_OUT_OPERATOR_FIL_13: 189
+   RECORDS_OUT_OPERATOR_FS_15: 189
+   RECORDS_OUT_OPERATOR_FS_21: 121
+   RECORDS_OUT_OPERATOR_GBY_17: 121
+   RECORDS_OUT_OPERATOR_GBY_19: 121
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_8: 121
-   RECORDS_OUT_OPERATOR_SEL_10: 121
-   RECORDS_OUT_OPERATOR_SEL_2: 189
-   RECORDS_OUT_OPERATOR_SEL_6: 189
+   RECORDS_OUT_OPERATOR_RS_18: 121
+   RECORDS_OUT_OPERATOR_SEL_14: 189
+   RECORDS_OUT_OPERATOR_SEL_16: 189
+   RECORDS_OUT_OPERATOR_SEL_20: 121
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 189
 Stage-1 INPUT COUNTERS:
@@ -94,16 +94,16 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2_n3: 189
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
-   RECORDS_OUT_OPERATOR_FIL_12: 189
-   RECORDS_OUT_OPERATOR_FS_11: 121
-   RECORDS_OUT_OPERATOR_FS_4: 189
-   RECORDS_OUT_OPERATOR_GBY_7: 121
-   RECORDS_OUT_OPERATOR_GBY_9: 121
+   RECORDS_OUT_OPERATOR_FIL_13: 189
+   RECORDS_OUT_OPERATOR_FS_15: 189
+   RECORDS_OUT_OPERATOR_FS_21: 121
+   RECORDS_OUT_OPERATOR_GBY_17: 121
+   RECORDS_OUT_OPERATOR_GBY_19: 121
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_8: 121
-   RECORDS_OUT_OPERATOR_SEL_10: 121
-   RECORDS_OUT_OPERATOR_SEL_2: 189
-   RECORDS_OUT_OPERATOR_SEL_6: 189
+   RECORDS_OUT_OPERATOR_RS_18: 121
+   RECORDS_OUT_OPERATOR_SEL_14: 189
+   RECORDS_OUT_OPERATOR_SEL_16: 189
+   RECORDS_OUT_OPERATOR_SEL_20: 121
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 189
 Stage-1 INPUT COUNTERS:
@@ -125,16 +125,16 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_1_default.src2_n3: 292
    RECORDS_OUT_INTERMEDIATE_Map_1: 184
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
-   RECORDS_OUT_OPERATOR_FIL_12: 292
-   RECORDS_OUT_OPERATOR_FS_11: 184
-   RECORDS_OUT_OPERATOR_FS_4: 292
-   RECORDS_OUT_OPERATOR_GBY_7: 184
-   RECORDS_OUT_OPERATOR_GBY_9: 184
+   RECORDS_OUT_OPERATOR_FIL_13: 292
+   RECORDS_OUT_OPERATOR_FS_15: 292
+   RECORDS_OUT_OPERATOR_FS_21: 184
+   RECORDS_OUT_OPERATOR_GBY_17: 184
+   RECORDS_OUT_OPERATOR_GBY_19: 184
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_8: 184
-   RECORDS_OUT_OPERATOR_SEL_10: 184
-   RECORDS_OUT_OPERATOR_SEL_2: 292
-   RECORDS_OUT_OPERATOR_SEL_6: 292
+   RECORDS_OUT_OPERATOR_RS_18: 184
+   RECORDS_OUT_OPERATOR_SEL_14: 292
+   RECORDS_OUT_OPERATOR_SEL_16: 292
+   RECORDS_OUT_OPERATOR_SEL_20: 184
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 292
 Stage-1 INPUT COUNTERS:
@@ -175,25 +175,25 @@ Stage-2 HIVE COUNTERS:
    RECORDS_OUT_INTERMEDIATE_Map_1: 121
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
    RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
-   RECORDS_OUT_OPERATOR_FIL_23: 84
-   RECORDS_OUT_OPERATOR_FIL_24: 105
-   RECORDS_OUT_OPERATOR_FS_11: 57
-   RECORDS_OUT_OPERATOR_FS_15: 105
-   RECORDS_OUT_OPERATOR_FS_22: 64
-   RECORDS_OUT_OPERATOR_FS_4: 84
-   RECORDS_OUT_OPERATOR_GBY_18: 64
-   RECORDS_OUT_OPERATOR_GBY_20: 64
-   RECORDS_OUT_OPERATOR_GBY_7: 57
-   RECORDS_OUT_OPERATOR_GBY_9: 57
+   RECORDS_OUT_OPERATOR_FIL_25: 84
+   RECORDS_OUT_OPERATOR_FIL_26: 105
+   RECORDS_OUT_OPERATOR_FS_29: 84
+   RECORDS_OUT_OPERATOR_FS_31: 105
+   RECORDS_OUT_OPERATOR_FS_39: 57
+   RECORDS_OUT_OPERATOR_FS_42: 64
+   RECORDS_OUT_OPERATOR_GBY_33: 57
+   RECORDS_OUT_OPERATOR_GBY_34: 64
+   RECORDS_OUT_OPERATOR_GBY_37: 57
+   RECORDS_OUT_OPERATOR_GBY_40: 64
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_19: 64
-   RECORDS_OUT_OPERATOR_RS_8: 57
-   RECORDS_OUT_OPERATOR_SEL_10: 57
-   RECORDS_OUT_OPERATOR_SEL_13: 105
-   RECORDS_OUT_OPERATOR_SEL_17: 105
-   RECORDS_OUT_OPERATOR_SEL_2: 84
-   RECORDS_OUT_OPERATOR_SEL_21: 64
-   RECORDS_OUT_OPERATOR_SEL_6: 84
+   RECORDS_OUT_OPERATOR_RS_35: 57
+   RECORDS_OUT_OPERATOR_RS_36: 64
+   RECORDS_OUT_OPERATOR_SEL_27: 84
+   RECORDS_OUT_OPERATOR_SEL_28: 105
+   RECORDS_OUT_OPERATOR_SEL_30: 84
+   RECORDS_OUT_OPERATOR_SEL_32: 105
+   RECORDS_OUT_OPERATOR_SEL_38: 57
+   RECORDS_OUT_OPERATOR_SEL_41: 64
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 189
 Stage-2 INPUT COUNTERS:
@@ -220,25 +220,25 @@ Stage-2 HIVE COUNTERS:
    RECORDS_OUT_INTERMEDIATE_Map_1: 184
    RECORDS_OUT_INTERMEDIATE_Reducer_2: 0
    RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
-   RECORDS_OUT_OPERATOR_FIL_23: 84
-   RECORDS_OUT_OPERATOR_FIL_24: 208
-   RECORDS_OUT_OPERATOR_FS_11: 57
-   RECORDS_OUT_OPERATOR_FS_15: 208
-   RECORDS_OUT_OPERATOR_FS_22: 127
-   RECORDS_OUT_OPERATOR_FS_4: 84
-   RECORDS_OUT_OPERATOR_GBY_18: 127
-   RECORDS_OUT_OPERATOR_GBY_20: 127
-   RECORDS_OUT_OPERATOR_GBY_7: 57
-   RECORDS_OUT_OPERATOR_GBY_9: 57
+   RECORDS_OUT_OPERATOR_FIL_25: 84
+   RECORDS_OUT_OPERATOR_FIL_26: 208
+   RECORDS_OUT_OPERATOR_FS_29: 84
+   RECORDS_OUT_OPERATOR_FS_31: 208
+   RECORDS_OUT_OPERATOR_FS_39: 57
+   RECORDS_OUT_OPERATOR_FS_42: 127
+   RECORDS_OUT_OPERATOR_GBY_33: 57
+   RECORDS_OUT_OPERATOR_GBY_34: 127
+   RECORDS_OUT_OPERATOR_GBY_37: 57
+   RECORDS_OUT_OPERATOR_GBY_40: 127
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_19: 127
-   RECORDS_OUT_OPERATOR_RS_8: 57
-   RECORDS_OUT_OPERATOR_SEL_10: 57
-   RECORDS_OUT_OPERATOR_SEL_13: 208
-   RECORDS_OUT_OPERATOR_SEL_17: 208
-   RECORDS_OUT_OPERATOR_SEL_2: 84
-   RECORDS_OUT_OPERATOR_SEL_21: 127
-   RECORDS_OUT_OPERATOR_SEL_6: 84
+   RECORDS_OUT_OPERATOR_RS_35: 57
+   RECORDS_OUT_OPERATOR_RS_36: 127
+   RECORDS_OUT_OPERATOR_SEL_27: 84
+   RECORDS_OUT_OPERATOR_SEL_28: 208
+   RECORDS_OUT_OPERATOR_SEL_30: 84
+   RECORDS_OUT_OPERATOR_SEL_32: 208
+   RECORDS_OUT_OPERATOR_SEL_38: 57
+   RECORDS_OUT_OPERATOR_SEL_41: 127
    RECORDS_OUT_OPERATOR_TS_0: 500
    TOTAL_TABLE_ROWS_WRITTEN: 292
 Stage-2 INPUT COUNTERS:
@@ -274,24 +274,24 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Map_4: 64
    RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
-   RECORDS_OUT_OPERATOR_FIL_20: 84
-   RECORDS_OUT_OPERATOR_FIL_29: 105
-   RECORDS_OUT_OPERATOR_FS_16: 121
-   RECORDS_OUT_OPERATOR_FS_24: 84
-   RECORDS_OUT_OPERATOR_FS_33: 105
-   RECORDS_OUT_OPERATOR_GBY_14: 121
-   RECORDS_OUT_OPERATOR_GBY_26: 57
-   RECORDS_OUT_OPERATOR_GBY_35: 64
+   RECORDS_OUT_OPERATOR_FIL_37: 84
+   RECORDS_OUT_OPERATOR_FIL_47: 105
+   RECORDS_OUT_OPERATOR_FS_40: 84
+   RECORDS_OUT_OPERATOR_FS_46: 121
+   RECORDS_OUT_OPERATOR_FS_50: 105
+   RECORDS_OUT_OPERATOR_GBY_42: 57
+   RECORDS_OUT_OPERATOR_GBY_44: 121
+   RECORDS_OUT_OPERATOR_GBY_52: 64
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_27: 57
-   RECORDS_OUT_OPERATOR_RS_36: 64
-   RECORDS_OUT_OPERATOR_SEL_15: 121
-   RECORDS_OUT_OPERATOR_SEL_21: 84
-   RECORDS_OUT_OPERATOR_SEL_23: 84
-   RECORDS_OUT_OPERATOR_SEL_25: 84
-   RECORDS_OUT_OPERATOR_SEL_30: 105
-   RECORDS_OUT_OPERATOR_SEL_32: 105
-   RECORDS_OUT_OPERATOR_SEL_34: 105
+   RECORDS_OUT_OPERATOR_RS_43: 57
+   RECORDS_OUT_OPERATOR_RS_53: 64
+   RECORDS_OUT_OPERATOR_SEL_38: 84
+   RECORDS_OUT_OPERATOR_SEL_39: 84
+   RECORDS_OUT_OPERATOR_SEL_41: 84
+   RECORDS_OUT_OPERATOR_SEL_45: 121
+   RECORDS_OUT_OPERATOR_SEL_48: 105
+   RECORDS_OUT_OPERATOR_SEL_49: 105
+   RECORDS_OUT_OPERATOR_SEL_51: 105
    RECORDS_OUT_OPERATOR_TS_19: 500
    RECORDS_OUT_OPERATOR_TS_28: 500
    TOTAL_TABLE_ROWS_WRITTEN: 189
@@ -324,24 +324,24 @@ Stage-1 HIVE COUNTERS:
    RECORDS_OUT_INTERMEDIATE_Map_1: 57
    RECORDS_OUT_INTERMEDIATE_Map_4: 127
    RECORDS_OUT_INTERMEDIATE_Reducer_3: 0
-   RECORDS_OUT_OPERATOR_FIL_20: 84
-   RECORDS_OUT_OPERATOR_FIL_29: 208
-   RECORDS_OUT_OPERATOR_FS_16: 184
-   RECORDS_OUT_OPERATOR_FS_24: 84
-   RECORDS_OUT_OPERATOR_FS_33: 208
-   RECORDS_OUT_OPERATOR_GBY_14: 184
-   RECORDS_OUT_OPERATOR_GBY_26: 57
-   RECORDS_OUT_OPERATOR_GBY_35: 127
+   RECORDS_OUT_OPERATOR_FIL_37: 84
+   RECORDS_OUT_OPERATOR_FIL_47: 208
+   RECORDS_OUT_OPERATOR_FS_40: 84
+   RECORDS_OUT_OPERATOR_FS_46: 184
+   RECORDS_OUT_OPERATOR_FS_50: 208
+   RECORDS_OUT_OPERATOR_GBY_42: 57
+   RECORDS_OUT_OPERATOR_GBY_44: 184
+   RECORDS_OUT_OPERATOR_GBY_52: 127
    RECORDS_OUT_OPERATOR_MAP_0: 0
-   RECORDS_OUT_OPERATOR_RS_27: 57
-   RECORDS_OUT_OPERATOR_RS_36: 127
-   RECORDS_OUT_OPERATOR_SEL_15: 184
-   RECORDS_OUT_OPERATOR_SEL_21: 84
-   RECORDS_OUT_OPERATOR_SEL_23: 84
-   RECORDS_OUT_OPERATOR_SEL_25: 84
-   RECORDS_OUT_OPERATOR_SEL_30: 208
-   RECORDS_OUT_OPERATOR_SEL_32: 208
-   RECORDS_OUT_OPERATOR_SEL_34: 208
+   RECORDS_OUT_OPERATOR_RS_43: 57
+   RECORDS_OUT_OPERATOR_RS_53: 127
+   RECORDS_OUT_OPERATOR_SEL_38: 84
+   RECORDS_OUT_OPERATOR_SEL_39: 84
+   RECORDS_OUT_OPERATOR_SEL_41: 84
+   RECORDS_OUT_OPERATOR_SEL_45: 184
+   RECORDS_OUT_OPERATOR_SEL_48: 208
+   RECORDS_OUT_OPERATOR_SEL_49: 208
+   RECORDS_OUT_OPERATOR_SEL_51: 208
    RECORDS_OUT_OPERATOR_TS_19: 500
    RECORDS_OUT_OPERATOR_TS_28: 500
    TOTAL_TABLE_ROWS_WRITTEN: 292
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out b/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
index b5b0414..2eb6efb 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_partition_pruning.q.out
@@ -116,7 +116,7 @@ STAGE PLANS:
                     outputColumnNames: col1, col2
                     Statistics: Num rows: 2 Data size: 736 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector(col1, 'hll'), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector(col2, 'hll')
+                      aggregations: max(length(col1)), avg(COALESCE(length(col1),0)), count(1), count(col1), compute_bit_vector_hll(col1), max(length(col2)), avg(COALESCE(length(col2),0)), count(col2), compute_bit_vector_hll(col2)
                       minReductionHashAggr: 0.5
                       mode: hash
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -130,7 +130,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: max(VALUE._col0), avg(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
diff --git a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
index f7b4439..7292d6a 100644
--- a/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynamic_semijoin_reduction_3.q.out
@@ -212,7 +212,7 @@ STAGE PLANS:
                       name: default.acidtbl
                   Write Type: DELETE
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int)
@@ -232,7 +232,7 @@ STAGE PLANS:
                   outputColumnNames: a, b
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
-                    aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll')
+                    aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(b), max(b), count(b), compute_bit_vector_hll(b)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -243,10 +243,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary)
         Reducer 5 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
@@ -491,7 +491,7 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col1 (type: int)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int)
@@ -511,7 +511,7 @@ STAGE PLANS:
                   outputColumnNames: a, b
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
-                    aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll')
+                    aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(b), max(b), count(b), compute_bit_vector_hll(b)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -522,10 +522,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary)
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
@@ -793,7 +793,7 @@ STAGE PLANS:
                       name: default.acidtbl
                   Write Type: DELETE
         Reducer 5 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: int), VALUE._col0 (type: int)
@@ -813,7 +813,7 @@ STAGE PLANS:
                   outputColumnNames: a, b
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
-                    aggregations: min(a), max(a), count(1), count(a), compute_bit_vector(a, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll')
+                    aggregations: min(a), max(a), count(1), count(a), compute_bit_vector_hll(a), min(b), max(b), count(b), compute_bit_vector_hll(b)
                     minReductionHashAggr: 0.99
                     mode: hash
                     outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
@@ -824,10 +824,10 @@ STAGE PLANS:
                       Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col0 (type: int), _col1 (type: int), _col2 (type: bigint), _col3 (type: bigint), _col4 (type: binary), _col5 (type: int), _col6 (type: int), _col7 (type: bigint), _col8 (type: binary)
         Reducer 6 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8
                 Statistics: Num rows: 1 Data size: 328 Basic stats: COMPLETE Column stats: NONE
diff --git a/ql/src/test/results/clientpositive/llap/dynpart_cast.q.out b/ql/src/test/results/clientpositive/llap/dynpart_cast.q.out
index 3ae3dd7..fe44004 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_cast.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_cast.q.out
@@ -60,7 +60,7 @@ STAGE PLANS:
                       outputColumnNames: i, static_part, dyn_part
                       Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: COMPLETE
                       Group By Operator
-                        aggregations: min(i), max(i), count(1), count(i), compute_bit_vector(i, 'hll')
+                        aggregations: min(i), max(i), count(1), count(i), compute_bit_vector_hll(i)
                         keys: static_part (type: int), dyn_part (type: int)
                         minReductionHashAggr: 0.4
                         mode: hash
@@ -76,10 +76,10 @@ STAGE PLANS:
             Execution mode: llap
             LLAP IO: no inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4)
                 keys: KEY._col0 (type: int), KEY._col1 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6
diff --git a/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_bucketing.q.out b/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_bucketing.q.out
index 2055f7e..739479e 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_bucketing.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_bucketing.q.out
@@ -431,7 +431,7 @@ STAGE PLANS:
                           outputColumnNames: ca_address_sk, ca_address_id, ca_street_number, ca_street_name, ca_street_type, ca_suite_number, ca_city, ca_county, ca_state, ca_zip, ca_country, ca_gmt_offset, ca_location_type
                           Statistics: Num rows: 1 Data size: 112 Basic stats: COMPLETE Column stats: COMPLETE
                           Group By Operator
-                            aggregations: min(ca_address_sk), max(ca_address_sk), count(1), count(ca_address_sk), compute_bit_vector(ca_address_sk, 'hll'), max(length(ca_address_id)), avg(COALESCE(length(ca_address_id),0)), count(ca_address_id), compute_bit_vector(ca_address_id, 'hll'), max(length(ca_street_number)), avg(COALESCE(length(ca_street_number),0)), count(ca_street_number), compute_bit_vector(ca_street_number, 'hll'), max(length(ca_street_name)), avg(COALESCE(length(ca_street_n [...]
+                            aggregations: min(ca_address_sk), max(ca_address_sk), count(1), count(ca_address_sk), compute_bit_vector_hll(ca_address_sk), max(length(ca_address_id)), avg(COALESCE(length(ca_address_id),0)), count(ca_address_id), compute_bit_vector_hll(ca_address_id), max(length(ca_street_number)), avg(COALESCE(length(ca_street_number),0)), count(ca_street_number), compute_bit_vector_hll(ca_street_number), max(length(ca_street_name)), avg(COALESCE(length(ca_street_name),0)), [...]
                             keys: ca_location_type (type: string)
                             minReductionHashAggr: 0.4
                             mode: hash
@@ -466,7 +466,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16), max(VALUE._col17), avg(VALUE._col18), count(VALUE._col19), compute_bit_vector(V [...]
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), max(VALUE._col5), avg(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), max(VALUE._col9), avg(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), max(VALUE._col13), avg(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16), max(VALUE._col17), avg(VALUE._col18), count(VALUE._col19), comp [...]
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25, _col26, _col27, _col28, _col29, _col30, _col31, _col32, _col33, _col34, _col35, _col36, _col37, _col38, _col39, _col40, _col41, _col42, _col43, _col44, _col45, _col46, _col47, _col48, _col49
diff --git a/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out b/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out
index 502717f..83c393e 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_sort_opt_vectorization.q.out
@@ -196,7 +196,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: smallint), VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: float), VALUE._col3 (type: tinyint)
@@ -214,7 +214,7 @@ STAGE PLANS:
                   outputColumnNames: si, i, b, f, ds, t
                   Statistics: Num rows: 11 Data size: 1221 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                     keys: ds (type: string), t (type: tinyint)
                     minReductionHashAggr: 0.5454545
                     mode: hash
@@ -243,10 +243,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.over1k_part_orc
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -335,7 +335,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 10
@@ -356,7 +356,7 @@ STAGE PLANS:
                     outputColumnNames: si, i, b, f, ds, t
                     Statistics: Num rows: 10 Data size: 1110 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                       keys: ds (type: string), t (type: tinyint)
                       minReductionHashAggr: 0.5
                       mode: hash
@@ -385,10 +385,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.over1k_part_limit_orc
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -477,7 +477,7 @@ STAGE PLANS:
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 11 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.5454545
                           mode: hash
@@ -490,7 +490,7 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 5 Data size: 3300 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: vectorized, llap
@@ -508,10 +508,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.over1k_part_buck_orc
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -599,7 +599,7 @@ STAGE PLANS:
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 11 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.5454545
                           mode: hash
@@ -612,7 +612,7 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 5 Data size: 3300 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: vectorized, llap
@@ -630,10 +630,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.over1k_part_buck_sort_orc
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -788,7 +788,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: KEY.reducesinkkey0 (type: smallint), VALUE._col0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: float), VALUE._col3 (type: tinyint)
@@ -806,7 +806,7 @@ STAGE PLANS:
                   outputColumnNames: si, i, b, f, ds, t
                   Statistics: Num rows: 11 Data size: 1221 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                     keys: ds (type: string), t (type: tinyint)
                     minReductionHashAggr: 0.5454545
                     mode: hash
@@ -835,10 +835,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.over1k_part_orc
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -927,7 +927,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 10
@@ -948,7 +948,7 @@ STAGE PLANS:
                     outputColumnNames: si, i, b, f, ds, t
                     Statistics: Num rows: 10 Data size: 1110 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                       keys: ds (type: string), t (type: tinyint)
                       minReductionHashAggr: 0.5
                       mode: hash
@@ -977,10 +977,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.over1k_part_limit_orc
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -1069,7 +1069,7 @@ STAGE PLANS:
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 11 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.5454545
                           mode: hash
@@ -1082,7 +1082,7 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 5 Data size: 3300 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: vectorized, llap
@@ -1100,10 +1100,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.over1k_part_buck_orc
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -1191,7 +1191,7 @@ STAGE PLANS:
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 11 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.5454545
                           mode: hash
@@ -1204,7 +1204,7 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 5 Data size: 3300 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: vectorized, llap
@@ -1222,10 +1222,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.ql.io.orc.OrcSerde
                       name: default.over1k_part_buck_sort_orc
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -1765,7 +1765,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: VALUE._col0 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: float), VALUE._col3 (type: tinyint)
@@ -1784,7 +1784,7 @@ STAGE PLANS:
                   outputColumnNames: si, i, b, f, ds, t
                   Statistics: Num rows: 11 Data size: 1221 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                     keys: ds (type: string), t (type: tinyint)
                     minReductionHashAggr: 0.5454545
                     mode: hash
@@ -1798,10 +1798,10 @@ STAGE PLANS:
                       Statistics: Num rows: 5 Data size: 3735 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col2 (type: smallint), _col3 (type: smallint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: int), _col9 (type: bigint), _col10 (type: binary), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: bigint), _col14 (type: binary), _col15 (type: float), _col16 (type: float), _col17 (type: bigint), _col18 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -1887,7 +1887,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: VALUE._col0 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: float), VALUE._col3 (type: tinyint)
@@ -1905,7 +1905,7 @@ STAGE PLANS:
                   outputColumnNames: si, i, b, f, ds, t
                   Statistics: Num rows: 11 Data size: 1221 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                     keys: ds (type: string), t (type: tinyint)
                     minReductionHashAggr: 0.5454545
                     mode: hash
@@ -1934,10 +1934,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                       name: default.over1k_part2_orc
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -2025,7 +2025,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: VALUE._col0 (type: tinyint), VALUE._col1 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float)
@@ -2053,7 +2053,7 @@ STAGE PLANS:
                         outputColumnNames: si, i, b, f, ds, t
                         Statistics: Num rows: 1 Data size: 111 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: ds (type: string), t (type: tinyint)
                           minReductionHashAggr: 0.4
                           mode: hash
@@ -2082,10 +2082,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                       name: default.over1k_part2_orc
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -2172,7 +2172,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
@@ -2196,7 +2196,7 @@ STAGE PLANS:
                     outputColumnNames: si, i, b, f, ds, t
                     Statistics: Num rows: 5 Data size: 555 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                       keys: ds (type: string), t (type: tinyint)
                       minReductionHashAggr: 0.6
                       mode: hash
@@ -2210,10 +2210,10 @@ STAGE PLANS:
                         Statistics: Num rows: 2 Data size: 1494 Basic stats: COMPLETE Column stats: COMPLETE
                         value expressions: _col2 (type: smallint), _col3 (type: smallint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: int), _col9 (type: bigint), _col10 (type: binary), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: bigint), _col14 (type: binary), _col15 (type: float), _col16 (type: float), _col17 (type: bigint), _col18 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -2300,7 +2300,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
                 keys: KEY._col0 (type: tinyint), KEY._col1 (type: smallint), KEY._col2 (type: int), KEY._col3 (type: bigint), KEY._col4 (type: float)
@@ -2316,7 +2316,7 @@ STAGE PLANS:
                     outputColumnNames: si, i, b, f, ds, t
                     Statistics: Num rows: 5 Data size: 555 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                       keys: ds (type: string), t (type: tinyint)
                       minReductionHashAggr: 0.6
                       mode: hash
@@ -2342,10 +2342,10 @@ STAGE PLANS:
                           serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                           name: default.over1k_part2_orc
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -2736,7 +2736,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Select Operator
                 expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY.reducesinkkey0 (type: float), VALUE._col3 (type: tinyint)
@@ -2755,7 +2755,7 @@ STAGE PLANS:
                   outputColumnNames: si, i, b, f, t
                   Statistics: Num rows: 11 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
                   Group By Operator
-                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                     keys: t (type: tinyint)
                     minReductionHashAggr: 0.5454545
                     mode: hash
@@ -2769,10 +2769,10 @@ STAGE PLANS:
                       Statistics: Num rows: 5 Data size: 3300 Basic stats: COMPLETE Column stats: COMPLETE
                       value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -2860,7 +2860,7 @@ STAGE PLANS:
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 11 Data size: 264 Basic stats: COMPLETE Column stats: COMPLETE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.5454545
                           mode: hash
@@ -2873,7 +2873,7 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 5 Data size: 3300 Basic stats: COMPLETE Column stats: COMPLETE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-            Execution mode: llap
+            Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: vectorized, llap
@@ -2891,10 +2891,10 @@ STAGE PLANS:
                       serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
                       name: default.over1k_part_buck_sort2_orc
         Reducer 3 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -3463,7 +3463,7 @@ STAGE PLANS:
             Execution mode: vectorized, llap
             LLAP IO: all inputs
         Reducer 2 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Limit
                 Number of rows: 10
@@ -3484,7 +3484,7 @@ STAGE PLANS:
                     outputColumnNames: i, si, s
                     Statistics: Num rows: 10 Data size: 816 Basic stats: COMPLETE Column stats: COMPLETE
                     Group By Operator
-                      aggregations: min(i), max(i), count(1), count(i), compute_bit_vector(i, 'hll'), min(si), max(si), count(si), compute_bit_vector(si, 'hll')
+                      aggregations: min(i), max(i), count(1), count(i), compute_bit_vector_hll(i), min(si), max(si), count(si), compute_bit_vector_hll(si)
                       keys: s (type: string)
                       minReductionHashAggr: 0.4
                       mode: hash
@@ -3514,10 +3514,10 @@ STAGE PLANS:
                       name: default.addcolumns_vectorization_true_disallowincompatible_true_fileformat_orc_tinyint
                   Write Type: INSERT
         Reducer 4 
-            Execution mode: llap
+            Execution mode: vectorized, llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8)
                 keys: KEY._col0 (type: string)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9
diff --git a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
index f1b40e8..3a63768 100644
--- a/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
+++ b/ql/src/test/results/clientpositive/llap/dynpart_sort_optimization.q.out
@@ -143,12 +143,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: tinyint)
+                        null sort order: a
+                        sort order: +
+                        Map-reduce partition columns: _col4 (type: tinyint)
+                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), 'foo' (type: string), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, ds, t
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: ds (type: string), t (type: tinyint)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -161,20 +168,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
                             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col2 (type: smallint), _col3 (type: smallint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: int), _col9 (type: bigint), _col10 (type: binary), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: bigint), _col14 (type: binary), _col15 (type: float), _col16 (type: float), _col17 (type: bigint), _col18 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col4 (type: tinyint)
-                        null sort order: a
-                        sort order: +
-                        Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -190,21 +205,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part
 
   Stage: Stage-2
     Dependency Collection
@@ -287,12 +287,19 @@ STAGE PLANS:
                   expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: tinyint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col4 (type: tinyint)
+                    null sort order: a
+                    sort order: +
+                    Map-reduce partition columns: _col4 (type: tinyint)
+                    Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
                   Select Operator
                     expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), 'foo' (type: string), _col4 (type: tinyint)
                     outputColumnNames: si, i, b, f, ds, t
                     Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                       keys: ds (type: string), t (type: tinyint)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -305,18 +312,26 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col2 (type: smallint), _col3 (type: smallint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: int), _col9 (type: bigint), _col10 (type: binary), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: bigint), _col14 (type: binary), _col15 (type: float), _col16 (type: float), _col17 (type: bigint), _col18 (type: binary)
-                  Reduce Output Operator
-                    key expressions: _col4 (type: tinyint)
-                    null sort order: a
-                    sort order: +
-                    Map-reduce partition columns: _col4 (type: tinyint)
-                    Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part_limit
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -332,21 +347,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part_limit
 
   Stage: Stage-2
     Dependency Collection
@@ -408,12 +408,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col0 (type: smallint)
+                        null sort order: aaa
+                        sort order: +++
+                        Map-reduce partition columns: _col4 (type: tinyint)
+                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -426,20 +433,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col0 (type: smallint)
-                        null sort order: aaa
-                        sort order: +++
-                        Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: KEY._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part_buck
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -455,21 +470,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_BUCKET_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part_buck
 
   Stage: Stage-2
     Dependency Collection
@@ -530,12 +530,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col3 (type: float)
+                        null sort order: aaa
+                        sort order: +++
+                        Map-reduce partition columns: _col4 (type: tinyint)
+                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -548,20 +555,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col3 (type: float)
-                        null sort order: aaa
-                        sort order: +++
-                        Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part_buck_sort
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -577,21 +592,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_BUCKET_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part_buck_sort
 
   Stage: Stage-2
     Dependency Collection
@@ -721,12 +721,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: tinyint)
+                        null sort order: a
+                        sort order: +
+                        Map-reduce partition columns: _col4 (type: tinyint)
+                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), 'foo' (type: string), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, ds, t
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: ds (type: string), t (type: tinyint)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -739,20 +746,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
                             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col2 (type: smallint), _col3 (type: smallint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: int), _col9 (type: bigint), _col10 (type: binary), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: bigint), _col14 (type: binary), _col15 (type: float), _col16 (type: float), _col17 (type: bigint), _col18 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col4 (type: tinyint)
-                        null sort order: a
-                        sort order: +
-                        Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -768,21 +783,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part
 
   Stage: Stage-2
     Dependency Collection
@@ -865,12 +865,19 @@ STAGE PLANS:
                   expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), VALUE._col4 (type: tinyint)
                   outputColumnNames: _col0, _col1, _col2, _col3, _col4
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  Reduce Output Operator
+                    key expressions: _col4 (type: tinyint)
+                    null sort order: a
+                    sort order: +
+                    Map-reduce partition columns: _col4 (type: tinyint)
+                    Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                    value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
                   Select Operator
                     expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), 'foo' (type: string), _col4 (type: tinyint)
                     outputColumnNames: si, i, b, f, ds, t
                     Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                       keys: ds (type: string), t (type: tinyint)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -883,18 +890,26 @@ STAGE PLANS:
                         Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         value expressions: _col2 (type: smallint), _col3 (type: smallint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: int), _col9 (type: bigint), _col10 (type: binary), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: bigint), _col14 (type: binary), _col15 (type: float), _col16 (type: float), _col17 (type: bigint), _col18 (type: binary)
-                  Reduce Output Operator
-                    key expressions: _col4 (type: tinyint)
-                    null sort order: a
-                    sort order: +
-                    Map-reduce partition columns: _col4 (type: tinyint)
-                    Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                    value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part_limit
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -910,21 +925,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part_limit
 
   Stage: Stage-2
     Dependency Collection
@@ -986,12 +986,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col0 (type: smallint)
+                        null sort order: aaa
+                        sort order: +++
+                        Map-reduce partition columns: _col4 (type: tinyint)
+                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -1004,20 +1011,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col0 (type: smallint)
-                        null sort order: aaa
-                        sort order: +++
-                        Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: KEY._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part_buck
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -1033,21 +1048,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: KEY._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_BUCKET_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part_buck
 
   Stage: Stage-2
     Dependency Collection
@@ -1108,12 +1108,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col3 (type: float)
+                        null sort order: aaa
+                        sort order: +++
+                        Map-reduce partition columns: _col4 (type: tinyint)
+                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -1126,20 +1133,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col3 (type: float)
-                        null sort order: aaa
-                        sort order: +++
-                        Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part_buck_sort
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -1155,21 +1170,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_BUCKET_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part_buck_sort
 
   Stage: Stage-2
     Dependency Collection
@@ -1713,7 +1713,7 @@ STAGE PLANS:
                   outputColumnNames: si, i, b, f, ds, t
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
-                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                     keys: ds (type: string), t (type: tinyint)
                     minReductionHashAggr: 0.99
                     mode: hash
@@ -1730,7 +1730,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -1822,12 +1822,19 @@ STAGE PLANS:
                 expressions: VALUE._col0 (type: smallint), KEY.reducesinkkey0 (type: int), VALUE._col1 (type: bigint), VALUE._col2 (type: float), VALUE._col3 (type: tinyint)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
                 Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                Reduce Output Operator
+                  key expressions: _col4 (type: tinyint), _col1 (type: int)
+                  null sort order: aa
+                  sort order: ++
+                  Map-reduce partition columns: _col4 (type: tinyint)
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  value expressions: _col0 (type: smallint), _col2 (type: bigint), _col3 (type: float)
                 Select Operator
                   expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), 'foo' (type: string), _col4 (type: tinyint)
                   outputColumnNames: si, i, b, f, ds, t
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
-                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                     keys: ds (type: string), t (type: tinyint)
                     minReductionHashAggr: 0.99
                     mode: hash
@@ -1840,18 +1847,26 @@ STAGE PLANS:
                       Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                       value expressions: _col2 (type: smallint), _col3 (type: smallint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: int), _col9 (type: bigint), _col10 (type: binary), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: bigint), _col14 (type: binary), _col15 (type: float), _col16 (type: float), _col17 (type: bigint), _col18 (type: binary)
-                Reduce Output Operator
-                  key expressions: _col4 (type: tinyint), _col1 (type: int)
-                  null sort order: aa
-                  sort order: ++
-                  Map-reduce partition columns: _col4 (type: tinyint)
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  value expressions: _col0 (type: smallint), _col2 (type: bigint), _col3 (type: float)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), KEY._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part2
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -1867,21 +1882,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), KEY._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part2
 
   Stage: Stage-2
     Dependency Collection
@@ -1970,12 +1970,19 @@ STAGE PLANS:
                       expressions: _col1 (type: smallint), _col2 (type: int), _col3 (type: bigint), _col4 (type: float), _col0 (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: tinyint)
+                        null sort order: a
+                        sort order: +
+                        Map-reduce partition columns: _col4 (type: tinyint)
+                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), 'foo' (type: string), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, ds, t
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: ds (type: string), t (type: tinyint)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -1988,36 +1995,9 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint)
                             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col2 (type: smallint), _col3 (type: smallint), _col4 (type: bigint), _col5 (type: bigint), _col6 (type: binary), _col7 (type: int), _col8 (type: int), _col9 (type: bigint), _col10 (type: binary), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: bigint), _col14 (type: binary), _col15 (type: float), _col16 (type: float), _col17 (type: bigint), _col18 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col4 (type: tinyint)
-                        null sort order: a
-                        sort order: +
-                        Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float)
         Reducer 3 
             Execution mode: llap
             Reduce Operator Tree:
-              Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
-                keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
-                mode: mergepartial
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
-                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                Select Operator
-                  expressions: 'LONG' (type: string), UDFToLong(_col2) (type: bigint), UDFToLong(_col3) (type: bigint), (_col4 - _col5) (type: bigint), COALESCE(ndv_compute_bit_vector(_col6),0) (type: bigint), _col6 (type: binary), 'LONG' (type: string), UDFToLong(_col7) (type: bigint), UDFToLong(_col8) (type: bigint), (_col4 - _col9) (type: bigint), COALESCE(ndv_compute_bit_vector(_col10),0) (type: bigint), _col10 (type: binary), 'LONG' (type: string), _col11 (type: bigint), _col12 (typ [...]
-                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  File Output Operator
-                    compressed: false
-                    Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                    table:
-                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
-                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
-                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 4 
-            Execution mode: llap
-            Reduce Operator Tree:
               Select Operator
                 expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), VALUE._col3 (type: float), KEY._col4 (type: tinyint)
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4
@@ -2025,11 +2005,31 @@ STAGE PLANS:
                   compressed: false
                   Dp Sort State: PARTITION_SORTED
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part2
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part2
+        Reducer 4 
+            Execution mode: llap
+            Reduce Operator Tree:
+              Group By Operator
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
+                keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
+                mode: mergepartial
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
+                Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                Select Operator
+                  expressions: 'LONG' (type: string), UDFToLong(_col2) (type: bigint), UDFToLong(_col3) (type: bigint), (_col4 - _col5) (type: bigint), COALESCE(ndv_compute_bit_vector(_col6),0) (type: bigint), _col6 (type: binary), 'LONG' (type: string), UDFToLong(_col7) (type: bigint), UDFToLong(_col8) (type: bigint), (_col4 - _col9) (type: bigint), COALESCE(ndv_compute_bit_vector(_col10),0) (type: bigint), _col10 (type: binary), 'LONG' (type: string), _col11 (type: bigint), _col12 (typ [...]
+                  outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18, _col19, _col20, _col21, _col22, _col23, _col24, _col25
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  File Output Operator
+                    compressed: false
+                    Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                    table:
+                        input format: org.apache.hadoop.mapred.SequenceFileInputFormat
+                        output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
+                        serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
 
   Stage: Stage-2
     Dependency Collection
@@ -2125,7 +2125,7 @@ STAGE PLANS:
                     outputColumnNames: si, i, b, f, ds, t
                     Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                       keys: ds (type: string), t (type: tinyint)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -2142,7 +2142,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -2245,7 +2245,7 @@ STAGE PLANS:
                     outputColumnNames: si, i, b, f, ds, t
                     Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                     Group By Operator
-                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                      aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                       keys: ds (type: string), t (type: tinyint)
                       minReductionHashAggr: 0.99
                       mode: hash
@@ -2274,7 +2274,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17, _col18
@@ -2684,7 +2684,7 @@ STAGE PLANS:
                   outputColumnNames: si, i, b, f, t
                   Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                   Group By Operator
-                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                    aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                     keys: t (type: tinyint)
                     minReductionHashAggr: 0.99
                     mode: hash
@@ -2701,7 +2701,7 @@ STAGE PLANS:
             Execution mode: llap
             Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -2777,12 +2777,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), i (type: int), b (type: bigint), f (type: float), t (type: tinyint)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4
                       Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col3 (type: float)
+                        null sort order: aaa
+                        sort order: +++
+                        Map-reduce partition columns: _col4 (type: tinyint)
+                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint), _col3 (type: float), _col4 (type: tinyint)
                         outputColumnNames: si, i, b, f, t
                         Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(i), max(i), count(i), compute_bit_vector(i, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(i), max(i), count(i), compute_bit_vector_hll(i), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: t (type: tinyint)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -2795,20 +2802,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: tinyint)
                             Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col1 (type: smallint), _col2 (type: smallint), _col3 (type: bigint), _col4 (type: bigint), _col5 (type: binary), _col6 (type: int), _col7 (type: int), _col8 (type: bigint), _col9 (type: binary), _col10 (type: bigint), _col11 (type: bigint), _col12 (type: bigint), _col13 (type: binary), _col14 (type: float), _col15 (type: float), _col16 (type: bigint), _col17 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col4 (type: tinyint), _bucket_number (type: string), _col3 (type: float)
-                        null sort order: aaa
-                        sort order: +++
-                        Map-reduce partition columns: _col4 (type: tinyint)
-                        Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: smallint), _col1 (type: int), _col2 (type: bigint)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_BUCKET_SORTED
+                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part_buck_sort2
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector(VALUE._col16)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12), min(VALUE._col13), max(VALUE._col14), count(VALUE._col15), compute_bit_vector_hll(VALUE._col16)
                 keys: KEY._col0 (type: tinyint)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15, _col16, _col17
@@ -2824,21 +2839,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: int), VALUE._col2 (type: bigint), KEY._col3 (type: float), KEY._col4 (type: tinyint), KEY._bucket_number (type: string)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _bucket_number
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_BUCKET_SORTED
-                  Statistics: Num rows: 1 Data size: 24 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part_buck_sort2
 
   Stage: Stage-2
     Dependency Collection
@@ -3195,12 +3195,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), b (type: bigint), f (type: float), 'foo' (type: string), t (type: tinyint), i (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
+                        null sort order: aaa
+                        sort order: +++
+                        Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
+                        Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
                         outputColumnNames: si, b, f, s, t, i
                         Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: s (type: string), t (type: tinyint), i (type: int)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -3213,20 +3220,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint), _col2 (type: int)
                             Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col3 (type: smallint), _col4 (type: smallint), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: binary), _col8 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col11 (type: binary), _col12 (type: float), _col13 (type: float), _col14 (type: bigint), _col15 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
-                        null sort order: aaa
-                        sort order: +++
-                        Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
-                        Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_SORTED
+                  Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part3
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint), KEY._col2 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15
@@ -3242,21 +3257,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-                      name: default.over1k_part3
 
   Stage: Stage-2
     Dependency Collection
@@ -3319,12 +3319,19 @@ STAGE PLANS:
                       expressions: si (type: smallint), b (type: bigint), f (type: float), s (type: string), 27Y (type: tinyint), i (type: int)
                       outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
                       Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+                      Reduce Output Operator
+                        key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
+                        null sort order: aaa
+                        sort order: +++
+                        Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
+                        Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+                        value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float)
                       Select Operator
                         expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float), _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
                         outputColumnNames: si, b, f, s, t, i
                         Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
                         Group By Operator
-                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector(si, 'hll'), min(b), max(b), count(b), compute_bit_vector(b, 'hll'), min(f), max(f), count(f), compute_bit_vector(f, 'hll')
+                          aggregations: min(si), max(si), count(1), count(si), compute_bit_vector_hll(si), min(b), max(b), count(b), compute_bit_vector_hll(b), min(f), max(f), count(f), compute_bit_vector_hll(f)
                           keys: s (type: string), t (type: tinyint), i (type: int)
                           minReductionHashAggr: 0.99
                           mode: hash
@@ -3337,20 +3344,28 @@ STAGE PLANS:
                             Map-reduce partition columns: _col0 (type: string), _col1 (type: tinyint), _col2 (type: int)
                             Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
                             value expressions: _col3 (type: smallint), _col4 (type: smallint), _col5 (type: bigint), _col6 (type: bigint), _col7 (type: binary), _col8 (type: bigint), _col9 (type: bigint), _col10 (type: bigint), _col11 (type: binary), _col12 (type: float), _col13 (type: float), _col14 (type: bigint), _col15 (type: binary)
-                      Reduce Output Operator
-                        key expressions: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
-                        null sort order: aaa
-                        sort order: +++
-                        Map-reduce partition columns: _col3 (type: string), _col4 (type: tinyint), _col5 (type: int)
-                        Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
-                        value expressions: _col0 (type: smallint), _col1 (type: bigint), _col2 (type: float)
             Execution mode: llap
             LLAP IO: all inputs
         Reducer 2 
             Execution mode: llap
             Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
+                File Output Operator
+                  compressed: false
+                  Dp Sort State: PARTITION_SORTED
+                  Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
+                  table:
+                      input format: org.apache.hadoop.mapred.TextInputFormat
+                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
+                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
+                      name: default.over1k_part3
+        Reducer 3 
+            Execution mode: llap
+            Reduce Operator Tree:
               Group By Operator
-                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector(VALUE._col12)
+                aggregations: min(VALUE._col0), max(VALUE._col1), count(VALUE._col2), count(VALUE._col3), compute_bit_vector_hll(VALUE._col4), min(VALUE._col5), max(VALUE._col6), count(VALUE._col7), compute_bit_vector_hll(VALUE._col8), min(VALUE._col9), max(VALUE._col10), count(VALUE._col11), compute_bit_vector_hll(VALUE._col12)
                 keys: KEY._col0 (type: string), KEY._col1 (type: tinyint), KEY._col2 (type: int)
                 mode: mergepartial
                 outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, _col6, _col7, _col8, _col9, _col10, _col11, _col12, _col13, _col14, _col15
@@ -3366,21 +3381,6 @@ STAGE PLANS:
                         input format: org.apache.hadoop.mapred.SequenceFileInputFormat
                         output format: org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat
                         serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
-        Reducer 3 
-            Execution mode: llap
-            Reduce Operator Tree:
-              Select Operator
-                expressions: VALUE._col0 (type: smallint), VALUE._col1 (type: bigint), VALUE._col2 (type: float), KEY._col3 (type: string), KEY._col4 (type: tinyint), KEY._col5 (type: int)
-                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5
-                File Output Operator
-                  compressed: false
-                  Dp Sort State: PARTITION_SORTED
-                  Statistics: Num rows: 1 Data size: 208 Basic stats: COMPLETE Column stats: NONE
-                  table:
-                      input format: org.apache.hadoop.mapred.TextInputFormat
-                      output format: org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat
-                      serde: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
... 27100 lines suppressed ...