Github user gatorsmile commented on a diff in the pull request:

    https://github.com/apache/spark/pull/20624#discussion_r170480388
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
    @@ -154,4 +156,120 @@ case class AnalyzeColumnCommand(
         AttributeMap(attributePercentiles.toSeq)
       }
     
    +  /** Returns true iff the we support gathering column statistics on 
column of the given type. */
    +  private def supportsType(dataType: DataType): Boolean = dataType match {
    +    case _: IntegralType => true
    +    case _: DecimalType => true
    +    case DoubleType | FloatType => true
    +    case BooleanType => true
    +    case DateType => true
    +    case TimestampType => true
    +    case BinaryType | StringType => true
    +    case _ => false
    +  }
    +
    +  /** Returns true iff the we support gathering histogram on column of the 
given type. */
    +  private def supportsHistogram(dataType: DataType): Boolean = dataType 
match {
    +    case _: IntegralType => true
    +    case _: DecimalType => true
    +    case DoubleType | FloatType => true
    +    case DateType => true
    +    case TimestampType => true
    +    case _ => false
    +  }
    +
    +  /**
    +   * Constructs an expression to compute column statistics for a given 
column.
    +   *
    +   * The expression should create a single struct column with the 
following schema:
    +   * distinctCount: Long, min: T, max: T, nullCount: Long, avgLen: Long, 
maxLen: Long,
    +   * distinctCountsForIntervals: Array[Long]
    +   *
    +   * Together with [[rowToColumnStat]], this function is used to create 
[[ColumnStat]] and
    +   * as a result should stay in sync with it.
    +   */
    +  private def statExprs(
    +    col: Attribute,
    +    conf: SQLConf,
    +    colPercentiles: AttributeMap[ArrayData]): CreateNamedStruct = {
    +    def struct(exprs: Expression*): CreateNamedStruct = 
CreateStruct(exprs.map { expr =>
    +      expr.transformUp { case af: AggregateFunction => 
af.toAggregateExpression() }
    +    })
    +    val one = Literal(1, LongType)
    +
    +    // the approximate ndv (num distinct value) should never be larger 
than the number of rows
    +    val numNonNulls = if (col.nullable) Count(col) else Count(one)
    +    val ndv = Least(Seq(HyperLogLogPlusPlus(col, conf.ndvMaxError), 
numNonNulls))
    +    val numNulls = Subtract(Count(one), numNonNulls)
    +    val defaultSize = Literal(col.dataType.defaultSize, LongType)
    +    val nullArray = Literal(null, ArrayType(LongType))
    +
    +    def fixedLenTypeStruct: CreateNamedStruct = {
    +      val genHistogram =
    +        supportsHistogram(col.dataType) && colPercentiles.contains(col)
    +      val intervalNdvsExpr = if (genHistogram) {
    +        ApproxCountDistinctForIntervals(col,
    +          Literal(colPercentiles(col), ArrayType(col.dataType)), 
conf.ndvMaxError)
    +      } else {
    +        nullArray
    +      }
    +      // For fixed width types, avg size should be the same as max size.
    +      struct(ndv, Cast(Min(col), col.dataType), Cast(Max(col), 
col.dataType), numNulls,
    +        defaultSize, defaultSize, intervalNdvsExpr)
    +    }
    +
    +    col.dataType match {
    +      case _: IntegralType => fixedLenTypeStruct
    +      case _: DecimalType => fixedLenTypeStruct
    +      case DoubleType | FloatType => fixedLenTypeStruct
    +      case BooleanType => fixedLenTypeStruct
    +      case DateType => fixedLenTypeStruct
    +      case TimestampType => fixedLenTypeStruct
    +      case BinaryType | StringType =>
    +        // For string and binary type, we don't compute min, max or 
histogram
    +        val nullLit = Literal(null, col.dataType)
    +        struct(
    +          ndv, nullLit, nullLit, numNulls,
    +          // Set avg/max size to default size if all the values are null 
or there is no value.
    +          Coalesce(Seq(Ceil(Average(Length(col))), defaultSize)),
    +          Coalesce(Seq(Cast(Max(Length(col)), LongType), defaultSize)),
    +          nullArray)
    +      case _ =>
    +        throw new AnalysisException("Analyzing column statistics is not 
supported for column " +
    +          s"${col.name} of data type: ${col.dataType}.")
    +    }
    +  }
    +
    +  /** Convert a struct for column stats (defined in `statExprs`) into 
[[ColumnStat]]. */
    +  private def rowToColumnStat(
    +    row: InternalRow,
    +    attr: Attribute,
    +    rowCount: Long,
    +    percentiles: Option[ArrayData]): ColumnStat = {
    +    // The first 6 fields are basic column stats, the 7th is ndvs for 
histogram bins.
    +    val cs = ColumnStat(
    +      distinctCount = Option(BigInt(row.getLong(0))),
    +      // for string/binary min/max, get should return null
    +      min = Option(row.get(1, attr.dataType)),
    +      max = Option(row.get(2, attr.dataType)),
    +      nullCount = Option(BigInt(row.getLong(3))),
    +      avgLen = Option(row.getLong(4)),
    +      maxLen = Option(row.getLong(5))
    +    )
    +    if (row.isNullAt(6) || !cs.nullCount.isDefined) {
    --- End diff --
    
    `!cs.nullCount.isDefined` -> `cs.nullCount.isEmpty`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to