You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by jc...@apache.org on 2020/07/20 02:52:27 UTC

[hive] branch master updated: HIVE-23730: TezCompiler tracking the original TS key columnName (Panagiotis Garefalakis, reviewed by Jesus Camacho Rodriguez)

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

jcamacho 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 2f735d1  HIVE-23730: TezCompiler tracking the original TS key columnName (Panagiotis Garefalakis, reviewed by Jesus Camacho Rodriguez)
2f735d1 is described below

commit 2f735d133349846799e47d27d60bf60b942c3f99
Author: Panagiotis Garefalakis <pg...@cloudera.com>
AuthorDate: Mon Jul 20 05:52:14 2020 +0300

    HIVE-23730: TezCompiler tracking the original TS key columnName (Panagiotis Garefalakis, reviewed by Jesus Camacho Rodriguez)
    
    Closes apache/hive#1152
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |  2 +-
 .../apache/hadoop/hive/ql/exec/OperatorUtils.java  | 35 +++++++++++
 .../apache/hadoop/hive/ql/parse/TezCompiler.java   | 28 +++++----
 .../results/clientpositive/llap/acid_mapjoin.q.out |  1 +
 .../llap/auto_sortmerge_join_13.q.out              |  3 +
 .../clientpositive/llap/bucket_map_join_tez2.q.out |  5 ++
 .../llap/bucket_map_join_tez_empty.q.out           |  1 +
 .../llap/bucketsortoptimize_insert_4.q.out         |  2 +
 .../llap/bucketsortoptimize_insert_5.q.out         |  2 +
 .../llap/bucketsortoptimize_insert_8.q.out         |  2 +
 .../llap/convert_decimal64_to_decimal.q.out        |  2 +
 .../llap/correlationoptimizer4.q.out               |  1 +
 .../llap/correlationoptimizer5.q.out               |  1 +
 .../clientpositive/llap/decimal_join2.q.out        |  1 +
 .../results/clientpositive/llap/hashjoin.q.out     |  3 +
 .../llap/hybridgrace_hashjoin_1.q.out              |  4 ++
 .../test/results/clientpositive/llap/join36.q.out  |  1 +
 .../clientpositive/llap/join_on_varchar.q.out      |  1 +
 .../results/clientpositive/llap/join_star.q.out    |  3 +
 .../llap/llap_vector_nohybridgrace.q.out           |  2 +
 .../results/clientpositive/llap/lvj_mapjoin.q.out  |  1 +
 .../clientpositive/llap/mapjoin_decimal.q.out      |  1 +
 .../llap/mapjoin_decimal_vectorized.q.out          |  4 ++
 .../llap/murmur_hash_migration.q.out               |  2 +
 .../results/clientpositive/llap/orc_llap.q.out     |  2 +
 .../results/clientpositive/llap/parquet_join.q.out |  2 +
 .../llap/probedecode_mapjoin_simple.q.out          |  2 +-
 .../llap/probedecode_mapjoin_stats.q.out           |  2 +-
 .../clientpositive/llap/skewjoin_mapjoin10.q.out   |  2 +
 .../llap/tez_bmj_schema_evolution.q.out            |  1 +
 .../clientpositive/llap/tez_nway_join.q.out        |  2 +
 .../results/clientpositive/llap/tez_smb_main.q.out |  6 ++
 .../llap/vector_binary_join_groupby.q.out          |  2 +
 .../clientpositive/llap/vector_char_mapjoin1.q.out |  2 +
 .../clientpositive/llap/vector_complex_join.q.out  |  1 +
 .../llap/vector_decimal64_multi_vertex.q.out       |  1 +
 .../clientpositive/llap/vector_decimal_join.q.out  |  1 +
 .../llap/vector_decimal_mapjoin.q.out              |  6 ++
 .../clientpositive/llap/vector_inner_join.q.out    |  9 +++
 .../llap/vector_leftsemi_mapjoin.q.out             | 70 ++++++++++++++++++++++
 .../clientpositive/llap/vector_llap_text_1.q.out   |  1 +
 .../llap/vector_mapjoin_reduce.q.out               |  1 +
 .../llap/vector_varchar_mapjoin1.q.out             |  2 +
 .../clientpositive/llap/vectorized_context.q.out   |  1 +
 .../clientpositive/llap/vectorized_mapjoin.q.out   |  1 +
 .../clientpositive/llap/vectorized_mapjoin2.q.out  |  1 +
 .../clientpositive/llap/vectorized_mapjoin3.q.out  |  3 +
 .../llap/vectorized_nested_mapjoin.q.out           |  1 +
 .../tez/hybridgrace_hashjoin_1.q.out               |  4 ++
 49 files changed, 221 insertions(+), 13 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index 28058b3..e6a188a 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -2547,7 +2547,7 @@ public class HiveConf extends Configuration {
         "The comma-separated list of SerDe classes that are considered when enhancing table-properties \n" +
             "during logical optimization."),
 
-    HIVE_OPTIMIZE_SCAN_PROBEDECODE("hive.optimize.scan.probedecode", false,
+    HIVE_OPTIMIZE_SCAN_PROBEDECODE("hive.optimize.scan.probedecode", true,
         "Whether to find suitable table scan operators that could reduce the number of decoded rows at runtime by probing extra available information. \n"
             + "The probe side for the row-level filtering is generated either statically in the case of expressions or dynamically for joins"
             + "e.g., use the cached MapJoin hashtable created on the small table side to filter out row columns that are not going "
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
index 2f114b8..6735a53 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/OperatorUtils.java
@@ -632,4 +632,39 @@ public class OperatorUtils {
     }
     return ret;
   }
+
+  /**
+   * Given an operator and an internalColName look for the original Table columnName
+   * that this internal column maps to. This method finds the original columnName
+   * by checking column Expr mappings and schemas of the start operator and its parents.
+   *
+   * @param start
+   * @param internalColName
+   * @return the original column name or null if not found
+   */
+  public static String findTableColNameOf(Operator<?> start, String internalColName) {
+    // Look for internalCoName alias in current OR Parent RowSchemas
+    Stack<Operator<?>> parentOps = new Stack<>();
+    ColumnInfo keyColInfo = null;
+    parentOps.add(start);
+    while (!parentOps.isEmpty()) {
+      Operator<?> currentOp = parentOps.pop();
+      if (currentOp instanceof ReduceSinkOperator) {
+        // Dont want to follow that parent path
+        continue;
+      }
+      // If columnName is the output of a ColumnExpr get the original columnName from the Expr Map
+      if (currentOp.getColumnExprMap() != null && currentOp.getColumnExprMap().containsKey(internalColName)
+              && currentOp.getColumnExprMap().get(internalColName) instanceof ExprNodeColumnDesc) {
+        internalColName = ((ExprNodeColumnDesc) currentOp.getColumnExprMap().get(internalColName)).getColumn();
+      }
+      keyColInfo = currentOp.getSchema().getColumnInfo(internalColName);
+      if (keyColInfo != null) {
+        // Get original colName alias (or fallback to internal colName)
+        return keyColInfo.getAlias() != null ? keyColInfo.getAlias() : keyColInfo.getInternalName();
+      }
+      parentOps.addAll(currentOp.getParentOperators());
+    }
+    return null;
+  }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
index ddcd022..f3ff485 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/TezCompiler.java
@@ -1512,19 +1512,22 @@ public class TezCompiler extends TaskCompiler {
           TableScanOperator.ProbeDecodeContext tsCntx = null;
           // Currently supporting: LowestRatio policy
           // TODO: Add more policies and make the selection a conf property
-          tsCntx = selectLowestRatioProbeDecodeMapJoin(probeTsMap.getKey(), probeTsMap.getValue());
-          LOG.debug("ProbeDecode MJ for TS {}  with CacheKey {} MJ Pos {} ColName {} with Ratio {}",
-              probeTsMap.getKey().getName(), tsCntx.getMjSmallTableCacheKey(), tsCntx.getMjSmallTablePos(),
-              tsCntx.getMjBigTableKeyColName(), tsCntx.getKeyRatio());
-          probeTsMap.getKey().setProbeDecodeContext(tsCntx);
-          probeTsMap.getKey().getConf().setProbeDecodeContext(tsCntx);
+          tsCntx = selectLowestRatioProbeDecodeMapJoin(probeTsMap.getKey(), probeTsMap.getValue(),
+                  procCtx.conf.getBoolVar(ConfVars.HIVE_IN_TEST));
+          if (tsCntx != null) {
+            LOG.debug("ProbeDecode MJ for TS {}  with CacheKey {} MJ Pos {} ColName {} with Ratio {}",
+                    probeTsMap.getKey().getName(), tsCntx.getMjSmallTableCacheKey(), tsCntx.getMjSmallTablePos(),
+                    tsCntx.getMjBigTableKeyColName(), tsCntx.getKeyRatio());
+            probeTsMap.getKey().setProbeDecodeContext(tsCntx);
+            probeTsMap.getKey().getConf().setProbeDecodeContext(tsCntx);
+          }
         }
       }
     }
   }
 
   private static TableScanOperator.ProbeDecodeContext selectLowestRatioProbeDecodeMapJoin(TableScanOperator tsOp,
-      List<MapJoinOperator> mjOps){
+      List<MapJoinOperator> mjOps, boolean inTestMode){
     MapJoinOperator selectedMJOp = null;
     double selectedMJOpRatio = 0;
     for (MapJoinOperator currMJOp : mjOps) {
@@ -1566,9 +1569,14 @@ public class TezCompiler extends TaskCompiler {
 
       List<ExprNodeDesc> keyDesc = selectedMJOp.getConf().getKeys().get(posBigTable);
       ExprNodeColumnDesc keyCol = (ExprNodeColumnDesc) keyDesc.get(0);
-
-      tsProbeDecodeCtx = new TableScanOperator.ProbeDecodeContext(mjCacheKey, mjSmallTablePos,
-          keyCol.getColumn(), selectedMJOpRatio);
+      String realTSColName = OperatorUtils.findTableColNameOf(selectedMJOp, keyCol.getColumn());
+      if (realTSColName != null) {
+        tsProbeDecodeCtx = new TableScanOperator.ProbeDecodeContext(mjCacheKey, mjSmallTablePos,
+                realTSColName, selectedMJOpRatio);
+      } else if (inTestMode){
+        throw new RuntimeException("ProbeDecode could not find TSColName for ColKey: " + keyCol + " with MJ Schema: " +
+                selectedMJOp.getSchema());
+      }
     }
     return tsProbeDecodeCtx;
   }
diff --git a/ql/src/test/results/clientpositive/llap/acid_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/acid_mapjoin.q.out
index 6b6a3a1..b05a5d7 100644
--- a/ql/src/test/results/clientpositive/llap/acid_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/acid_mapjoin.q.out
@@ -88,6 +88,7 @@ STAGE PLANS:
                 TableScan
                   alias: acid1
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:26.25
                   Statistics: Num rows: 316 Data size: 1264 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
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 018b6a5..761f491 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
@@ -97,6 +97,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.6
                   Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -404,6 +405,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.6
                   Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -711,6 +713,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.6
                   Statistics: Num rows: 10 Data size: 930 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
index 54082b0..b80be34 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez2.q.out
@@ -519,6 +519,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:value, smallTablePos:0, keyRatio:0.764
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: UDFToDouble(key) is not null (type: boolean)
@@ -610,6 +611,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: UDFToDouble(key) is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:value, smallTablePos:0, keyRatio:0.764
                   Statistics: Num rows: 500 Data size: 89000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: UDFToDouble(key) is not null (type: boolean)
@@ -768,6 +770,7 @@ STAGE PLANS:
                 TableScan
                   alias: tab_part_n11
                   filterExpr: (key > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.202
                   Statistics: Num rows: 500 Data size: 2000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 2) (type: boolean)
@@ -1319,6 +1322,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.5
                   Statistics: Num rows: 242 Data size: 968 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1984,6 +1988,7 @@ STAGE PLANS:
                 TableScan
                   alias: big
                   filterExpr: i is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:i, smallTablePos:0, keyRatio:1.1
                   Statistics: Num rows: 10 Data size: 40 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: i is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez_empty.q.out b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez_empty.q.out
index 8802231..fad819b 100644
--- a/ql/src/test/results/clientpositive/llap/bucket_map_join_tez_empty.q.out
+++ b/ql/src/test/results/clientpositive/llap/bucket_map_join_tez_empty.q.out
@@ -61,6 +61,7 @@ STAGE PLANS:
                 TableScan
                   alias: tab1_n0
                   filterExpr: key1 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:key1, smallTablePos:1, keyRatio:0.6666666666666666
                   Statistics: Num rows: 3 Data size: 273 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key1 is not null (type: boolean)
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 a60d24c..8656700 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
@@ -117,6 +117,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.07142857142857142
                   Statistics: Num rows: 84 Data size: 7896 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -373,6 +374,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.07142857142857142
                   Statistics: Num rows: 84 Data size: 336 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
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 736d049..72957fb 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
@@ -117,6 +117,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.07142857142857142
                   Statistics: Num rows: 84 Data size: 7896 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -360,6 +361,7 @@ STAGE PLANS:
                 TableScan
                   alias: test_table2_n8
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.07142857142857142
                   Statistics: Num rows: 84 Data size: 7896 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
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 aa00fa8..204f14e 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
@@ -117,6 +117,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.07142857142857142
                   Statistics: Num rows: 84 Data size: 7896 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -355,6 +356,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.07142857142857142
                   Statistics: Num rows: 84 Data size: 7896 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out b/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out
index 145ef94..58da99a 100644
--- a/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out
+++ b/ql/src/test/results/clientpositive/llap/convert_decimal64_to_decimal.q.out
@@ -161,6 +161,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: decimal0801_col_4 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:decimal0801_col_4, smallTablePos:1, keyRatio:0.6833333333333333
                   Statistics: Num rows: 1080 Data size: 109984 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -383,6 +384,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: decimal0801_col_4 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:decimal0801_col_4, smallTablePos:1, keyRatio:0.6833333333333333
                   Statistics: Num rows: 1080 Data size: 109984 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
index 527416f..95ba473 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer4.q.out
@@ -548,6 +548,7 @@ STAGE PLANS:
                 TableScan
                   alias: x
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_57_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/correlationoptimizer5.q.out b/ql/src/test/results/clientpositive/llap/correlationoptimizer5.q.out
index 41d77cd..1f506bd 100644
--- a/ql/src/test/results/clientpositive/llap/correlationoptimizer5.q.out
+++ b/ql/src/test/results/clientpositive/llap/correlationoptimizer5.q.out
@@ -600,6 +600,7 @@ STAGE PLANS:
                 TableScan
                   alias: n
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_76_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 1 Data size: 188 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/decimal_join2.q.out b/ql/src/test/results/clientpositive/llap/decimal_join2.q.out
index 0f07f09..0a3015e 100644
--- a/ql/src/test/results/clientpositive/llap/decimal_join2.q.out
+++ b/ql/src/test/results/clientpositive/llap/decimal_join2.q.out
@@ -245,6 +245,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.7368421052631579
                   Statistics: Num rows: 38 Data size: 4408 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/hashjoin.q.out b/ql/src/test/results/clientpositive/llap/hashjoin.q.out
index b95e69a..9acbd2d 100644
--- a/ql/src/test/results/clientpositive/llap/hashjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/hashjoin.q.out
@@ -102,6 +102,7 @@ STAGE PLANS:
                 TableScan
                   alias: x
                   filterExpr: cint is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_23_container, bigKeyColName:cint, smallTablePos:1, keyRatio:3.3333333333333335
                   Statistics: Num rows: 9 Data size: 927 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: cint is not null (type: boolean)
@@ -208,6 +209,7 @@ STAGE PLANS:
                 TableScan
                   alias: x
                   filterExpr: cvarchar is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_23_container, bigKeyColName:cvarchar, smallTablePos:1, keyRatio:3.3333333333333335
                   Statistics: Num rows: 9 Data size: 927 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: cvarchar is not null (type: boolean)
@@ -502,6 +504,7 @@ STAGE PLANS:
                 TableScan
                   alias: x
                   filterExpr: cdouble is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_23_container, bigKeyColName:cdouble, smallTablePos:1, keyRatio:2.0
                   Statistics: Num rows: 9 Data size: 927 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: cdouble is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
index 009881f..1d923ac 100644
--- a/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/hybridgrace_hashjoin_1.q.out
@@ -45,6 +45,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: (cint < 2000000000) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:2.0131022135416665
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (cint < 2000000000) (type: boolean)
@@ -175,6 +176,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: (cint < 2000000000) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:2.0131022135416665
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (cint < 2000000000) (type: boolean)
@@ -303,6 +305,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: cint is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:1.121826171875
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: cint is not null (type: boolean)
@@ -429,6 +432,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: cint is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:1.121826171875
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: cint is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/join36.q.out b/ql/src/test/results/clientpositive/llap/join36.q.out
index 95067e5..f35b804 100644
--- a/ql/src/test/results/clientpositive/llap/join36.q.out
+++ b/ql/src/test/results/clientpositive/llap/join36.q.out
@@ -82,6 +82,7 @@ STAGE PLANS:
                 TableScan
                   alias: x
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_32_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.9805825242718447
                   Statistics: Num rows: 309 Data size: 2472 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/join_on_varchar.q.out b/ql/src/test/results/clientpositive/llap/join_on_varchar.q.out
index 20d8b0f..2743e9a 100644
--- a/ql/src/test/results/clientpositive/llap/join_on_varchar.q.out
+++ b/ql/src/test/results/clientpositive/llap/join_on_varchar.q.out
@@ -103,6 +103,7 @@ STAGE PLANS:
                 TableScan
                   alias: tbl2_n2
                   filterExpr: c2 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:c1, smallTablePos:0, keyRatio:1.0
                   Statistics: Num rows: 2 Data size: 194 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: c2 is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/join_star.q.out b/ql/src/test/results/clientpositive/llap/join_star.q.out
index 16d539c..5ea3a70 100644
--- a/ql/src/test/results/clientpositive/llap/join_star.q.out
+++ b/ql/src/test/results/clientpositive/llap/join_star.q.out
@@ -153,6 +153,7 @@ STAGE PLANS:
                 TableScan
                   alias: fact
                   filterExpr: d1 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:d1, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 1 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: d1 is not null (type: boolean)
@@ -262,6 +263,7 @@ STAGE PLANS:
                 TableScan
                   alias: fact
                   filterExpr: (d1 is not null and d2 is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_45_container, bigKeyColName:d1, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 1 Data size: 16 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (d1 is not null and d2 is not null) (type: boolean)
@@ -405,6 +407,7 @@ STAGE PLANS:
                 TableScan
                   alias: dim1
                   filterExpr: (f1 is not null and f2 is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_45_container, bigKeyColName:f2, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 1 Data size: 8 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: (f1 is not null and f2 is not null) (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out b/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out
index a4cb5c5..7623cfe 100644
--- a/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out
+++ b/ql/src/test/results/clientpositive/llap/llap_vector_nohybridgrace.q.out
@@ -40,6 +40,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: (cint < 2000000000) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:2.0131022135416665
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -229,6 +230,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: (cint < 2000000000) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:2.0131022135416665
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out
index 37a50bb..6e0ef18 100644
--- a/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/lvj_mapjoin.q.out
@@ -121,6 +121,7 @@ STAGE PLANS:
                 TableScan
                   alias: expod1
                   filterExpr: aid is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_39_container, bigKeyColName:aid, smallTablePos:1, keyRatio:2.0
                   Statistics: Num rows: 3 Data size: 5772 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: aid is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
index 2dbc6da..4537607 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin_decimal.q.out
@@ -102,6 +102,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n95
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:dec, smallTablePos:1, keyRatio:0.09532888465204957
                   Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: dec is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/mapjoin_decimal_vectorized.q.out b/ql/src/test/results/clientpositive/llap/mapjoin_decimal_vectorized.q.out
index b90e4f1..08479fe 100644
--- a/ql/src/test/results/clientpositive/llap/mapjoin_decimal_vectorized.q.out
+++ b/ql/src/test/results/clientpositive/llap/mapjoin_decimal_vectorized.q.out
@@ -104,6 +104,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n95
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:dec, smallTablePos:1, keyRatio:0.09532888465204957
                   Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -526,6 +527,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n95
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:dec, smallTablePos:1, keyRatio:0.09532888465204957
                   Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -720,6 +722,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n95
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:dec, smallTablePos:1, keyRatio:1.1243680485338725
                   Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -1136,6 +1139,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n95
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:dec, smallTablePos:1, keyRatio:1.1243680485338725
                   Statistics: Num rows: 1049 Data size: 117488 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/murmur_hash_migration.q.out b/ql/src/test/results/clientpositive/llap/murmur_hash_migration.q.out
index 54600bc..e45db2c 100644
--- a/ql/src/test/results/clientpositive/llap/murmur_hash_migration.q.out
+++ b/ql/src/test/results/clientpositive/llap/murmur_hash_migration.q.out
@@ -596,6 +596,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.62
                   Statistics: Num rows: 150 Data size: 14250 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
@@ -870,6 +871,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.62
                   Statistics: Num rows: 150 Data size: 14250 Basic stats: COMPLETE Column stats: COMPLETE
                   GatherStats: false
                   Filter Operator
diff --git a/ql/src/test/results/clientpositive/llap/orc_llap.q.out b/ql/src/test/results/clientpositive/llap/orc_llap.q.out
index a0c8de7..1bd4fe7 100644
--- a/ql/src/test/results/clientpositive/llap/orc_llap.q.out
+++ b/ql/src/test/results/clientpositive/llap/orc_llap.q.out
@@ -632,6 +632,7 @@ STAGE PLANS:
                 TableScan
                   alias: o2
                   filterExpr: (csmallint is not null and cbigint is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_30_container, bigKeyColName:csmallint, smallTablePos:0, keyRatio:1.2950032552083333
                   Statistics: Num rows: 122880 Data size: 9906988 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (csmallint is not null and cbigint is not null) (type: boolean)
@@ -1123,6 +1124,7 @@ STAGE PLANS:
                 TableScan
                   alias: o2
                   filterExpr: (csmallint is not null and cbigint is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_30_container, bigKeyColName:csmallint, smallTablePos:0, keyRatio:2.5900146484375
                   Statistics: Num rows: 245760 Data size: 19813868 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (csmallint is not null and cbigint is not null) (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/parquet_join.q.out b/ql/src/test/results/clientpositive/llap/parquet_join.q.out
index 9525ebb..8ca7bd4 100644
--- a/ql/src/test/results/clientpositive/llap/parquet_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/parquet_join.q.out
@@ -215,6 +215,7 @@ STAGE PLANS:
                 TableScan
                   alias: p2
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:key, smallTablePos:0, keyRatio:1.0
                   Statistics: Num rows: 2 Data size: 198 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -351,6 +352,7 @@ STAGE PLANS:
                 TableScan
                   alias: p2
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:key, smallTablePos:0, keyRatio:1.0
                   Statistics: Num rows: 2 Data size: 204 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/probedecode_mapjoin_simple.q.out b/ql/src/test/results/clientpositive/llap/probedecode_mapjoin_simple.q.out
index cd4dafb..c716636 100644
--- a/ql/src/test/results/clientpositive/llap/probedecode_mapjoin_simple.q.out
+++ b/ql/src/test/results/clientpositive/llap/probedecode_mapjoin_simple.q.out
@@ -131,7 +131,7 @@ STAGE PLANS:
                 TableScan
                   alias: orders_fact
                   filterExpr: key2 is not null (type: boolean)
-                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:_col0, smallTablePos:1, keyRatio:1.0
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:key2, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 6 Data size: 264 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/probedecode_mapjoin_stats.q.out b/ql/src/test/results/clientpositive/llap/probedecode_mapjoin_stats.q.out
index 5fa8ea2..88455bd 100644
--- a/ql/src/test/results/clientpositive/llap/probedecode_mapjoin_stats.q.out
+++ b/ql/src/test/results/clientpositive/llap/probedecode_mapjoin_stats.q.out
@@ -191,7 +191,7 @@ STAGE PLANS:
                 TableScan
                   alias: orders_fact
                   filterExpr: (key2 is not null and key3 is not null) (type: boolean)
-                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_46_container, bigKeyColName:_col0, smallTablePos:1, keyRatio:0.16666666666666666
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_46_container, bigKeyColName:key3, smallTablePos:1, keyRatio:0.16666666666666666
                   Statistics: Num rows: 6 Data size: 288 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/skewjoin_mapjoin10.q.out b/ql/src/test/results/clientpositive/llap/skewjoin_mapjoin10.q.out
index 3c433ea..e07d166 100644
--- a/ql/src/test/results/clientpositive/llap/skewjoin_mapjoin10.q.out
+++ b/ql/src/test/results/clientpositive/llap/skewjoin_mapjoin10.q.out
@@ -95,6 +95,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.8333333333333334
                   Statistics: Num rows: 6 Data size: 540 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -300,6 +301,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.1666666666666667
                   Statistics: Num rows: 6 Data size: 24 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out b/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
index 4dd61fe..faf36c2 100644
--- a/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_bmj_schema_evolution.q.out
@@ -97,6 +97,7 @@ STAGE PLANS:
                 TableScan
                   alias: test_n1
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.303
                   Statistics: Num rows: 1000 Data size: 95000 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/tez_nway_join.q.out b/ql/src/test/results/clientpositive/llap/tez_nway_join.q.out
index 625e65c..ad23f7e 100644
--- a/ql/src/test/results/clientpositive/llap/tez_nway_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_nway_join.q.out
@@ -60,6 +60,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_46_container, bigKeyColName:key, smallTablePos:1, keyRatio:2.0
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -185,6 +186,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_52_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 3 Data size: 12 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out b/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
index 65aa8ed..26248d1 100644
--- a/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
+++ b/ql/src/test/results/clientpositive/llap/tez_smb_main.q.out
@@ -305,6 +305,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:0, keyRatio:1.044
                   Statistics: Num rows: 500 Data size: 1904 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -431,6 +432,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:0, keyRatio:1.044
                   Statistics: Num rows: 500 Data size: 1904 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -557,6 +559,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:0, keyRatio:1.1
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1659,6 +1662,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_50_container, bigKeyColName:key, smallTablePos:0, keyRatio:1.1
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1798,6 +1802,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_50_container, bigKeyColName:key, smallTablePos:0, keyRatio:1.1
                   Statistics: Num rows: 500 Data size: 5312 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1908,6 +1913,7 @@ STAGE PLANS:
                 TableScan
                   alias: s1
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_59_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0454545454545454
                   Statistics: Num rows: 242 Data size: 924 Basic stats: COMPLETE Column stats: NONE
                   Filter Operator
                     predicate: key is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out b/ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out
index ae9922e..cdc21f5 100644
--- a/ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_binary_join_groupby.q.out
@@ -131,6 +131,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: bin is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_30_container, bigKeyColName:bin, smallTablePos:1, keyRatio:0.0
                   Statistics: Num rows: 100 Data size: 34084 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -535,6 +536,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: i is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:i, smallTablePos:1, keyRatio:0.8
                   Statistics: Num rows: 100 Data size: 6000 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out b/ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out
index 26302c5..70d764b 100644
--- a/ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_char_mapjoin1.q.out
@@ -154,6 +154,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: c2 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:c2, smallTablePos:1, keyRatio:0.6666666666666666
                   Statistics: Num rows: 3 Data size: 273 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -332,6 +333,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: c2 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:c2, smallTablePos:1, keyRatio:0.6666666666666666
                   Statistics: Num rows: 3 Data size: 273 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out b/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
index 713f24a..4db6e49 100644
--- a/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_complex_join.q.out
@@ -51,6 +51,7 @@ STAGE PLANS:
                 TableScan
                   alias: alltypesorc
                   filterExpr: cint is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:cint, smallTablePos:1, keyRatio:0.8211263020833334
                   Statistics: Num rows: 12288 Data size: 2844090 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal64_multi_vertex.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal64_multi_vertex.q.out
index 6acc8a9..13e3cdb 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal64_multi_vertex.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal64_multi_vertex.q.out
@@ -115,6 +115,7 @@ STAGE PLANS:
                 TableScan
                   alias: store_sales
                   filterExpr: ss_item_sk is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:ss_item_sk, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_join.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_join.q.out
index 8f1283a..63a5c87 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_join.q.out
@@ -46,6 +46,7 @@ STAGE PLANS:
                 TableScan
                   alias: foo
                   filterExpr: x is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:x, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 1 Data size: 116 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
index c40d787..010d966 100644
--- a/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_decimal_mapjoin.q.out
@@ -107,6 +107,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n48
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:dec, smallTablePos:1, keyRatio:1.0448045757864632
                   Statistics: Num rows: 1049 Data size: 111776 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
@@ -392,6 +393,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_n48
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:dec, smallTablePos:1, keyRatio:1.0448045757864632
                   Statistics: Num rows: 1049 Data size: 223552 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
@@ -756,6 +758,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_small
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:dec, smallTablePos:1, keyRatio:1.0448045757864632
                   Statistics: Num rows: 1049 Data size: 111776 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
@@ -1042,6 +1045,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_small
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:dec, smallTablePos:1, keyRatio:1.0448045757864632
                   Statistics: Num rows: 1049 Data size: 223552 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
@@ -1331,6 +1335,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_small
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:dec, smallTablePos:1, keyRatio:1.0448045757864632
                   Statistics: Num rows: 1049 Data size: 111776 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
@@ -1618,6 +1623,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1_small
                   filterExpr: dec is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:dec, smallTablePos:1, keyRatio:1.0448045757864632
                   Statistics: Num rows: 1049 Data size: 223552 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_inner_join.q.out b/ql/src/test/results/clientpositive/llap/vector_inner_join.q.out
index 6bf8600..2b49ed1 100644
--- a/ql/src/test/results/clientpositive/llap/vector_inner_join.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_inner_join.q.out
@@ -65,6 +65,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:c, smallTablePos:1, keyRatio:0.4
                   Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -241,6 +242,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:c, smallTablePos:1, keyRatio:0.2
                   Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -459,6 +461,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:c, smallTablePos:1, keyRatio:0.4
                   Statistics: Num rows: 5 Data size: 20 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -692,6 +695,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:v1, smallTablePos:0, keyRatio:0.4
                   Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -810,6 +814,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:c, smallTablePos:1, keyRatio:0.4
                   Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -992,6 +997,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:c, smallTablePos:1, keyRatio:0.4
                   Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -1172,6 +1178,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:c, smallTablePos:1, keyRatio:0.4
                   Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -1406,6 +1413,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:v1, smallTablePos:0, keyRatio:0.4
                   Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -1586,6 +1594,7 @@ STAGE PLANS:
                 TableScan
                   alias: t2
                   filterExpr: (c > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:v1, smallTablePos:0, keyRatio:0.4
                   Statistics: Num rows: 5 Data size: 460 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out
index 27f2617..0393c66 100644
--- a/ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_leftsemi_mapjoin.q.out
@@ -162,6 +162,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -285,6 +286,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -410,6 +412,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0909090909090908
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -779,6 +782,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 5) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.5454545454545454
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 5) (type: boolean)
@@ -1024,6 +1028,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.18181818181818182
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 2) (type: boolean)
@@ -1146,6 +1151,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1282,6 +1288,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1405,6 +1412,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_52_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.6363636363636364
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1704,6 +1712,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_54_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -1876,6 +1885,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_53_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.22727272727272727
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -2590,6 +2600,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -2772,6 +2783,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -2970,6 +2982,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -3167,6 +3180,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -3456,6 +3470,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -3580,6 +3595,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -3706,6 +3722,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0909090909090908
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -4078,6 +4095,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 5) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.5454545454545454
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 5) (type: boolean)
@@ -4325,6 +4343,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.18181818181818182
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (key > 2) (type: boolean)
@@ -4448,6 +4467,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -4585,6 +4605,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -4709,6 +4730,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_52_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.6363636363636364
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -5011,6 +5033,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_54_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -5185,6 +5208,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_53_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.22727272727272727
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -5731,6 +5755,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -5915,6 +5940,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -6114,6 +6140,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -6258,6 +6285,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: key is not null (type: boolean)
@@ -9737,6 +9765,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -9926,6 +9955,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -10117,6 +10147,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0909090909090908
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -10689,6 +10720,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 5) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.5454545454545454
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -11073,6 +11105,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.18181818181818182
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -11261,6 +11294,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -11463,6 +11497,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -11653,6 +11688,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_52_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.6363636363636364
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -12112,6 +12148,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_54_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -12383,6 +12420,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_53_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.22727272727272727
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -13205,6 +13243,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -13481,6 +13520,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -13766,6 +13806,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -14050,6 +14091,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -14486,6 +14528,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -14672,6 +14715,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -14860,6 +14904,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0909090909090908
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -15425,6 +15470,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 5) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.5454545454545454
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -15804,6 +15850,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.18181818181818182
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -15989,6 +16036,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -16188,6 +16236,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -16375,6 +16424,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_52_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.6363636363636364
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -16825,6 +16875,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_54_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -17090,6 +17141,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_53_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.22727272727272727
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -17894,6 +17946,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -18164,6 +18217,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -18446,6 +18500,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -18727,6 +18782,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -19154,6 +19210,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -19341,6 +19398,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -19530,6 +19588,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:1.0909090909090908
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -20098,6 +20157,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 5) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.5454545454545454
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -20479,6 +20539,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: (key > 2) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.18181818181818182
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -20665,6 +20726,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -20865,6 +20927,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.36363636363636365
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -21053,6 +21116,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_52_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.6363636363636364
                   Statistics: Num rows: 11 Data size: 1023 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -21506,6 +21570,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_54_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -21773,6 +21838,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_53_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.22727272727272727
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -22583,6 +22649,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -22855,6 +22922,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -23138,6 +23206,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_34_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 88 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
@@ -23420,6 +23489,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_42_container, bigKeyColName:key, smallTablePos:1, keyRatio:0.13636363636363635
                   Statistics: Num rows: 22 Data size: 2046 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out b/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out
index ec8a181..fbdf3f6 100644
--- a/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_llap_text_1.q.out
@@ -208,6 +208,7 @@ STAGE PLANS:
                 TableScan
                   alias: b
                   filterExpr: key is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:key, smallTablePos:0, keyRatio:0.292
                   Statistics: Num rows: 500 Data size: 47500 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out b/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out
index b432dff..2ab579f 100644
--- a/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_mapjoin_reduce.q.out
@@ -318,6 +318,7 @@ STAGE PLANS:
                 TableScan
                   alias: li
                   filterExpr: ((l_linenumber = 1) and l_partkey is not null and l_orderkey is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_50_container, bigKeyColName:l_partkey, smallTablePos:1, keyRatio:0.14
                   Statistics: Num rows: 100 Data size: 1600 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vector_varchar_mapjoin1.q.out b/ql/src/test/results/clientpositive/llap/vector_varchar_mapjoin1.q.out
index f1e85f3..00c1c0f 100644
--- a/ql/src/test/results/clientpositive/llap/vector_varchar_mapjoin1.q.out
+++ b/ql/src/test/results/clientpositive/llap/vector_varchar_mapjoin1.q.out
@@ -154,6 +154,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: c2 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:c2, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: c2 is not null (type: boolean)
@@ -290,6 +291,7 @@ STAGE PLANS:
                 TableScan
                   alias: a
                   filterExpr: c2 is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_27_container, bigKeyColName:c2, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 3 Data size: 276 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: c2 is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_context.q.out b/ql/src/test/results/clientpositive/llap/vectorized_context.q.out
index a0241ad..58846e3 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_context.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_context.q.out
@@ -125,6 +125,7 @@ STAGE PLANS:
                 TableScan
                   alias: store_sales
                   filterExpr: (ss_store_sk is not null and ss_hdemo_sk is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_46_container, bigKeyColName:ss_hdemo_sk, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 6075 Data size: 72744 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (ss_store_sk is not null and ss_hdemo_sk is not null) (type: boolean)
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out
index 22822bf..891aa20 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin.q.out
@@ -32,6 +32,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: cint is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_30_container, bigKeyColName:cint, smallTablePos:1, keyRatio:0.4967447916666667
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out
index c330092..24cbd0f 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin2.q.out
@@ -66,6 +66,7 @@ STAGE PLANS:
                 TableScan
                   alias: x
                   filterExpr: a is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:a, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out
index d81a2be..fdf311b 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_mapjoin3.q.out
@@ -196,6 +196,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: (decimal0801_col is not null and int_col_1 is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:int_col_1, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
@@ -409,6 +410,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: (int_col_1 is not null and decimal0801_col is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:int_col_1, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
@@ -622,6 +624,7 @@ STAGE PLANS:
                 TableScan
                   alias: t1
                   filterExpr: (decimal0801_col is not null and int_col_1 is not null) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_25_container, bigKeyColName:int_col_1, smallTablePos:1, keyRatio:1.0
                   Statistics: Num rows: 5 Data size: 580 Basic stats: COMPLETE Column stats: NONE
                   TableScan Vectorization:
                       native: true
diff --git a/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out b/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out
index e946e94..2215135 100644
--- a/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out
+++ b/ql/src/test/results/clientpositive/llap/vectorized_nested_mapjoin.q.out
@@ -28,6 +28,7 @@ STAGE PLANS:
                 TableScan
                   alias: v3
                   filterExpr: csmallint is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_49_container, bigKeyColName:csmallint, smallTablePos:1, keyRatio:0.010579427083333334
                   Statistics: Num rows: 12288 Data size: 36700 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: csmallint is not null (type: boolean)
diff --git a/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out b/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out
index 34f63b3..586a0b6 100644
--- a/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out
+++ b/ql/src/test/results/clientpositive/tez/hybridgrace_hashjoin_1.q.out
@@ -45,6 +45,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: (cint < 2000000000) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:2.0131022135416665
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (cint < 2000000000) (type: boolean)
@@ -173,6 +174,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: (cint < 2000000000) (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:2.0131022135416665
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: (cint < 2000000000) (type: boolean)
@@ -300,6 +302,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: cint is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:1.121826171875
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: cint is not null (type: boolean)
@@ -424,6 +427,7 @@ STAGE PLANS:
                 TableScan
                   alias: c
                   filterExpr: cint is not null (type: boolean)
+                  probeDecodeDetails: cacheKey:HASH_MAP_MAPJOIN_29_container, bigKeyColName:cint, smallTablePos:1, keyRatio:1.121826171875
                   Statistics: Num rows: 12288 Data size: 36696 Basic stats: COMPLETE Column stats: COMPLETE
                   Filter Operator
                     predicate: cint is not null (type: boolean)