[ 
https://issues.apache.org/jira/browse/HIVE-26221?focusedWorklogId=832670&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-832670
 ]

ASF GitHub Bot logged work on HIVE-26221:
-----------------------------------------

                Author: ASF GitHub Bot
            Created on: 12/Dec/22 07:55
            Start Date: 12/Dec/22 07:55
    Worklog Time Spent: 10m 
      Work Description: amansinha100 commented on code in PR #3137:
URL: https://github.com/apache/hive/pull/3137#discussion_r1045449998


##########
ql/src/java/org/apache/hadoop/hive/ql/exec/DDLPlanUtils.java:
##########
@@ -395,29 +404,54 @@ public void addDoubleStats(ColumnStatisticsData cd, 
List<String> ls) {
     ls.add(lowValue + dc.getLowValue() + "'");
   }
 
+  public String checkHistogram(ColumnStatisticsData cd) {

Review Comment:
   nit: Strictly speaking, this is checking for the presence of the KLL sketch 
rather than the actual Histogram. The corresponding method for NDV is called 
checkBitVectors() instead of checkNDV().  At the same time, Histogram is more 
intuitive so I am ok with leaving this as-is and just add a comment for the 
method.



##########
common/src/java/org/apache/hadoop/hive/conf/HiveConf.java:
##########
@@ -2824,6 +2824,12 @@ public static enum ConfVars {
                     "higher compute cost. (NDV means the number of distinct 
values.). It only affects the FM-Sketch \n" +
                     "(not the HLL algorithm which is the default), where it 
computes the number of necessary\n" +
                     " bitvectors to achieve the accuracy."),
+    HIVE_STATS_KLL_K("hive.stats.kll.k", 200,

Review Comment:
   The single letter suffix 'k' comes across somewhat abrupt.  Can it be 
something like 'kparameter'  or 'kvalue' ?  



##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java:
##########
@@ -1004,41 +1045,51 @@ private long evaluateComparator(Statistics stats, 
AnnotateStatsProcCtx aspCtx, E
         closedBound = isClosedBound(udf);
       } else {
         // default
-        return numRows / 3;
+        return currNumRows / 3;
       }
 
       ColStatistics cs = 
stats.getColumnStatisticsFromColName(columnDesc.getColumn());
+      String colTypeLowerCase = columnDesc.getTypeString().toLowerCase();
+
+      if (FilterSelectivityEstimator.isHistogramAvailable(cs)) {
+        try {
+          return evaluateComparatorWithHistogram(
+              cs, currNumRows, colTypeLowerCase, boundValue, upperBound, 
closedBound);
+        } catch (SketchesArgumentException e) {
+          LOG.info("Sketch-based statistics estimation failed, falling back to 
regular estimation", e);

Review Comment:
   nit: adding a delimiter such as ":" or just a whitespace between the log 
message and the exception helps in readability of the logs. 



##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java:
##########
@@ -1234,17 +1285,70 @@ private long evaluateComparator(Statistics stats, 
AnnotateStatsProcCtx aspCtx, E
                 // new estimate for the number of rows
                 return Math.round(
                     
((maxValue.subtract(value)).divide(maxValue.subtract(minValue), 
RoundingMode.UP))
-                        .multiply(BigDecimal.valueOf(numRows))
+                        .multiply(BigDecimal.valueOf(currNumRows))
                         .doubleValue());
               }
             }
           }
         } catch (NumberFormatException nfe) {
-          return numRows / 3;
+          return currNumRows / 3;
         }
       }
       // default
-      return numRows / 3;
+      return currNumRows / 3;
+    }
+
+    private long evaluateComparatorWithHistogram(ColStatistics cs, long 
currNumRows, String colTypeLowerCase,
+        String boundValue, boolean upperBound, boolean closedBound) {
+      final KllFloatsSketch kll = 
KllFloatsSketch.heapify(Memory.wrap(cs.getHistogram()));
+
+      if (kll.getN() == 0) {
+        return 0;
+      }
+
+      try {
+        final float value = extractFloatFromLiteralValue(colTypeLowerCase, 
boundValue);
+
+        // kll ignores null values (i.e., kll.getN() + numNulls = 
currNumRows), we therefore need to use kll.getN()
+        // instead of currNumRows since the CDF is expressed as a fraction of 
kll.getN(), not currNumRows
+        if (upperBound) {
+          return Math.round(kll.getN() * (closedBound ?
+              lessThanOrEqualSelectivity(kll, value) : 
lessThanSelectivity(kll, value)));
+        } else {
+          return Math.round(kll.getN() * (closedBound ?
+              greaterThanOrEqualSelectivity(kll, value) : 
greaterThanSelectivity(kll, value)));
+        }
+      } catch (RuntimeException e) {
+        LOG.debug("Selectivity computation using histogram failed to parse the 
boundary value ({}), "
+            + ", using the generic computation strategy", boundValue, e);

Review Comment:
   nit: same comment as before about separation between the log message and 
exception. 



##########
ql/src/java/org/apache/hadoop/hive/ql/ddl/ShowUtils.java:
##########
@@ -209,6 +221,46 @@ public static String 
convertToString(org.apache.hadoop.hive.metastore.api.Date v
     return writableValue.toString();
   }
 
+  private static String convertHistogram(byte[] buffer, 
ColumnStatisticsData._Fields field) {

Review Comment:
   nit: A brief comment for this method would be useful.



##########
ql/src/test/results/clientpositive/beeline/colstats_all_nulls.q.out:
##########
@@ -73,6 +74,7 @@ max_col_len
 num_trues      
 num_falses     
 bit_vector     HL
+histogram      

Review Comment:
   The fact that the DESCRIBE FORMATTED output is changing by default makes me 
think whether it is better to not show this field if histogram stats is 
disabled.  Since this is brand new (and experimental) feature, the DESCRIBE 
output could be made conditional on whether it is enabled.  Perhaps this was 
discussed with other reviewers .. I am not sure.  



##########
ql/src/java/org/apache/hadoop/hive/ql/optimizer/stats/annotation/StatsRulesProcFactory.java:
##########
@@ -1234,17 +1285,70 @@ private long evaluateComparator(Statistics stats, 
AnnotateStatsProcCtx aspCtx, E
                 // new estimate for the number of rows
                 return Math.round(
                     
((maxValue.subtract(value)).divide(maxValue.subtract(minValue), 
RoundingMode.UP))
-                        .multiply(BigDecimal.valueOf(numRows))
+                        .multiply(BigDecimal.valueOf(currNumRows))
                         .doubleValue());
               }
             }
           }
         } catch (NumberFormatException nfe) {
-          return numRows / 3;
+          return currNumRows / 3;
         }
       }
       // default
-      return numRows / 3;
+      return currNumRows / 3;
+    }
+
+    private long evaluateComparatorWithHistogram(ColStatistics cs, long 
currNumRows, String colTypeLowerCase,
+        String boundValue, boolean upperBound, boolean closedBound) {
+      final KllFloatsSketch kll = 
KllFloatsSketch.heapify(Memory.wrap(cs.getHistogram()));
+
+      if (kll.getN() == 0) {
+        return 0;
+      }
+
+      try {
+        final float value = extractFloatFromLiteralValue(colTypeLowerCase, 
boundValue);
+
+        // kll ignores null values (i.e., kll.getN() + numNulls = 
currNumRows), we therefore need to use kll.getN()
+        // instead of currNumRows since the CDF is expressed as a fraction of 
kll.getN(), not currNumRows
+        if (upperBound) {
+          return Math.round(kll.getN() * (closedBound ?
+              lessThanOrEqualSelectivity(kll, value) : 
lessThanSelectivity(kll, value)));
+        } else {
+          return Math.round(kll.getN() * (closedBound ?
+              greaterThanOrEqualSelectivity(kll, value) : 
greaterThanSelectivity(kll, value)));
+        }
+      } catch (RuntimeException e) {
+        LOG.debug("Selectivity computation using histogram failed to parse the 
boundary value ({}), "
+            + ", using the generic computation strategy", boundValue, e);
+        return currNumRows / 3;
+      }
+    }
+
+    @VisibleForTesting
+    protected static float extractFloatFromLiteralValue(String 
colTypeLowerCase, String value) {
+      if (colTypeLowerCase.equals(serdeConstants.TINYINT_TYPE_NAME)) {
+        return Byte.parseByte(value);
+      } else if (colTypeLowerCase.equals(serdeConstants.SMALLINT_TYPE_NAME)) {
+        return Short.parseShort(value);
+      } else if (colTypeLowerCase.equals(serdeConstants.INT_TYPE_NAME)) {
+        return Integer.parseInt(value);
+      } else if (colTypeLowerCase.equals(serdeConstants.BIGINT_TYPE_NAME)) {
+        return Long.parseLong(value);
+      } else if (colTypeLowerCase.equals(serdeConstants.FLOAT_TYPE_NAME)) {
+        return Float.parseFloat(value);
+      } else if (colTypeLowerCase.equals(serdeConstants.DOUBLE_TYPE_NAME)) {
+        return (float) Double.parseDouble(value);
+      } else if 
(colTypeLowerCase.startsWith(serdeConstants.DECIMAL_TYPE_NAME)) {
+        return new BigDecimal(value).floatValue();

Review Comment:
   A Decimal value should be represented through a double rather than float 
since Decimal precision can exceed what float can store.  





Issue Time Tracking
-------------------

    Worklog Id:     (was: 832670)
    Time Spent: 9h 20m  (was: 9h 10m)

> Add histogram-based column statistics
> -------------------------------------
>
>                 Key: HIVE-26221
>                 URL: https://issues.apache.org/jira/browse/HIVE-26221
>             Project: Hive
>          Issue Type: Improvement
>          Components: CBO, Metastore, Statistics
>    Affects Versions: 4.0.0-alpha-2
>            Reporter: Alessandro Solimando
>            Assignee: Alessandro Solimando
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 9h 20m
>  Remaining Estimate: 0h
>
> Hive does not support histogram statistics, which are particularly useful for 
> skewed data (which is very common in practice) and range predicates.
> Hive's current selectivity estimation for range predicates is based on a 
> hard-coded value of 1/3 (see 
> [FilterSelectivityEstimator.java#L138-L144|https://github.com/apache/hive/blob/56c336268ea8c281d23c22d89271af37cb7e2572/ql/src/java/org/apache/hadoop/hive/ql/optimizer/calcite/stats/FilterSelectivityEstimator.java#L138-L144]).])
> The current proposal aims at integrating histogram as an additional column 
> statistics, stored into the Hive metastore at the table (or partition) level.
> The main requirements for histogram integration are the following:
>  * efficiency: the approach must scale and support billions of rows
>  * merge-ability: partition-level histograms have to be merged to form 
> table-level histograms
>  * explicit and configurable trade-off between memory footprint and accuracy
> Hive already integrates [KLL data 
> sketches|https://datasketches.apache.org/docs/KLL/KLLSketch.html] UDAF. 
> Datasketches are small, stateful programs that process massive data-streams 
> and can provide approximate answers, with mathematical guarantees, to 
> computationally difficult queries orders-of-magnitude faster than 
> traditional, exact methods.
> We propose to use KLL, and more specifically the cumulative distribution 
> function (CDF), as the underlying data structure for our histogram statistics.
> The current proposal targets numeric data types (float, integer and numeric 
> families) and temporal data types (date and timestamp).



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to