aboutsummaryrefslogtreecommitdiff
path: root/sql/core/src/test/scala
diff options
context:
space:
mode:
authorZhenhua Wang <wzh_zju@163.com>2016-10-03 10:12:02 -0700
committerReynold Xin <rxin@databricks.com>2016-10-03 10:12:02 -0700
commit7bf92127643570e4eb3610fa3ffd36839eba2718 (patch)
tree14386f49f956e97b50a8d6b2bbf0f776eab4dd39 /sql/core/src/test/scala
parenta27033c0bbaae8f31db9b91693947ed71738ed11 (diff)
downloadspark-7bf92127643570e4eb3610fa3ffd36839eba2718.tar.gz
spark-7bf92127643570e4eb3610fa3ffd36839eba2718.tar.bz2
spark-7bf92127643570e4eb3610fa3ffd36839eba2718.zip
[SPARK-17073][SQL] generate column-level statistics
## What changes were proposed in this pull request? Generate basic column statistics for all the atomic types: - numeric types: max, min, num of nulls, ndv (number of distinct values) - date/timestamp types: they are also represented as numbers internally, so they have the same stats as above. - string: avg length, max length, num of nulls, ndv - binary: avg length, max length, num of nulls - boolean: num of nulls, num of trues, num of falsies Also support storing and loading these statistics. One thing to notice: We support analyzing columns independently, e.g.: sql1: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key;` sql2: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS value;` when running sql2 to collect column stats for `value`, we don’t remove stats of columns `key` which are analyzed in sql1 and not in sql2. As a result, **users need to guarantee consistency** between sql1 and sql2. If the table has been changed before sql2, users should re-analyze column `key` when they want to analyze column `value`: `ANALYZE TABLE src COMPUTE STATISTICS FOR COLUMNS key, value;` ## How was this patch tested? add unit tests Author: Zhenhua Wang <wzh_zju@163.com> Closes #15090 from wzhfy/colStats.
Diffstat (limited to 'sql/core/src/test/scala')
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/StatisticsColumnSuite.scala334
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala16
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/StatisticsTest.scala129
3 files changed, 465 insertions, 14 deletions
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsColumnSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsColumnSuite.scala
new file mode 100644
index 0000000000..0ee0547c45
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsColumnSuite.scala
@@ -0,0 +1,334 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import java.sql.{Date, Timestamp}
+
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.parser.ParseException
+import org.apache.spark.sql.catalyst.plans.logical.ColumnStat
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.command.AnalyzeColumnCommand
+import org.apache.spark.sql.test.SQLTestData.ArrayData
+import org.apache.spark.sql.types._
+
+class StatisticsColumnSuite extends StatisticsTest {
+ import testImplicits._
+
+ test("parse analyze column commands") {
+ val tableName = "tbl"
+
+ // we need to specify column names
+ intercept[ParseException] {
+ sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS")
+ }
+
+ val analyzeSql = s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS key, value"
+ val parsed = spark.sessionState.sqlParser.parsePlan(analyzeSql)
+ val expected = AnalyzeColumnCommand(TableIdentifier(tableName), Seq("key", "value"))
+ comparePlans(parsed, expected)
+ }
+
+ test("analyzing columns of non-atomic types is not supported") {
+ val tableName = "tbl"
+ withTable(tableName) {
+ Seq(ArrayData(Seq(1, 2, 3), Seq(Seq(1, 2, 3)))).toDF().write.saveAsTable(tableName)
+ val err = intercept[AnalysisException] {
+ sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS FOR COLUMNS data")
+ }
+ assert(err.message.contains("Analyzing columns is not supported"))
+ }
+ }
+
+ test("check correctness of columns") {
+ val table = "tbl"
+ val colName1 = "abc"
+ val colName2 = "x.yz"
+ withTable(table) {
+ sql(s"CREATE TABLE $table ($colName1 int, `$colName2` string) USING PARQUET")
+
+ val invalidColError = intercept[AnalysisException] {
+ sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS key")
+ }
+ assert(invalidColError.message == "Invalid column name: key.")
+
+ withSQLConf("spark.sql.caseSensitive" -> "true") {
+ val invalidErr = intercept[AnalysisException] {
+ sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS ${colName1.toUpperCase}")
+ }
+ assert(invalidErr.message == s"Invalid column name: ${colName1.toUpperCase}.")
+ }
+
+ withSQLConf("spark.sql.caseSensitive" -> "false") {
+ val columnsToAnalyze = Seq(colName2.toUpperCase, colName1, colName2)
+ val tableIdent = TableIdentifier(table, Some("default"))
+ val relation = spark.sessionState.catalog.lookupRelation(tableIdent)
+ val (_, columnStats) =
+ AnalyzeColumnCommand(tableIdent, columnsToAnalyze).computeColStats(spark, relation)
+ assert(columnStats.contains(colName1))
+ assert(columnStats.contains(colName2))
+ // check deduplication
+ assert(columnStats.size == 2)
+ assert(!columnStats.contains(colName2.toUpperCase))
+ }
+ }
+ }
+
+ private def getNonNullValues[T](values: Seq[Option[T]]): Seq[T] = {
+ values.filter(_.isDefined).map(_.get)
+ }
+
+ test("column-level statistics for integral type columns") {
+ val values = (0 to 5).map { i =>
+ if (i % 2 == 0) None else Some(i)
+ }
+ val data = values.map { i =>
+ (i.map(_.toByte), i.map(_.toShort), i.map(_.toInt), i.map(_.toLong))
+ }
+
+ val df = data.toDF("c1", "c2", "c3", "c4")
+ val nonNullValues = getNonNullValues[Int](values)
+ val expectedColStatsSeq = df.schema.map { f =>
+ val colStat = ColumnStat(InternalRow(
+ values.count(_.isEmpty).toLong,
+ nonNullValues.max,
+ nonNullValues.min,
+ nonNullValues.distinct.length.toLong))
+ (f, colStat)
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("column-level statistics for fractional type columns") {
+ val values: Seq[Option[Decimal]] = (0 to 5).map { i =>
+ if (i == 0) None else Some(Decimal(i + i * 0.01))
+ }
+ val data = values.map { i =>
+ (i.map(_.toFloat), i.map(_.toDouble), i)
+ }
+
+ val df = data.toDF("c1", "c2", "c3")
+ val nonNullValues = getNonNullValues[Decimal](values)
+ val numNulls = values.count(_.isEmpty).toLong
+ val ndv = nonNullValues.distinct.length.toLong
+ val expectedColStatsSeq = df.schema.map { f =>
+ val colStat = f.dataType match {
+ case floatType: FloatType =>
+ ColumnStat(InternalRow(numNulls, nonNullValues.max.toFloat, nonNullValues.min.toFloat,
+ ndv))
+ case doubleType: DoubleType =>
+ ColumnStat(InternalRow(numNulls, nonNullValues.max.toDouble, nonNullValues.min.toDouble,
+ ndv))
+ case decimalType: DecimalType =>
+ ColumnStat(InternalRow(numNulls, nonNullValues.max, nonNullValues.min, ndv))
+ }
+ (f, colStat)
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("column-level statistics for string column") {
+ val values = Seq(None, Some("a"), Some("bbbb"), Some("cccc"), Some(""))
+ val df = values.toDF("c1")
+ val nonNullValues = getNonNullValues[String](values)
+ val expectedColStatsSeq = df.schema.map { f =>
+ val colStat = ColumnStat(InternalRow(
+ values.count(_.isEmpty).toLong,
+ nonNullValues.map(_.length).sum / nonNullValues.length.toDouble,
+ nonNullValues.map(_.length).max.toLong,
+ nonNullValues.distinct.length.toLong))
+ (f, colStat)
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("column-level statistics for binary column") {
+ val values = Seq(None, Some("a"), Some("bbbb"), Some("cccc"), Some("")).map(_.map(_.getBytes))
+ val df = values.toDF("c1")
+ val nonNullValues = getNonNullValues[Array[Byte]](values)
+ val expectedColStatsSeq = df.schema.map { f =>
+ val colStat = ColumnStat(InternalRow(
+ values.count(_.isEmpty).toLong,
+ nonNullValues.map(_.length).sum / nonNullValues.length.toDouble,
+ nonNullValues.map(_.length).max.toLong))
+ (f, colStat)
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("column-level statistics for boolean column") {
+ val values = Seq(None, Some(true), Some(false), Some(true))
+ val df = values.toDF("c1")
+ val nonNullValues = getNonNullValues[Boolean](values)
+ val expectedColStatsSeq = df.schema.map { f =>
+ val colStat = ColumnStat(InternalRow(
+ values.count(_.isEmpty).toLong,
+ nonNullValues.count(_.equals(true)).toLong,
+ nonNullValues.count(_.equals(false)).toLong))
+ (f, colStat)
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("column-level statistics for date column") {
+ val values = Seq(None, Some("1970-01-01"), Some("1970-02-02")).map(_.map(Date.valueOf))
+ val df = values.toDF("c1")
+ val nonNullValues = getNonNullValues[Date](values)
+ val expectedColStatsSeq = df.schema.map { f =>
+ val colStat = ColumnStat(InternalRow(
+ values.count(_.isEmpty).toLong,
+ // Internally, DateType is represented as the number of days from 1970-01-01.
+ nonNullValues.map(DateTimeUtils.fromJavaDate).max,
+ nonNullValues.map(DateTimeUtils.fromJavaDate).min,
+ nonNullValues.distinct.length.toLong))
+ (f, colStat)
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("column-level statistics for timestamp column") {
+ val values = Seq(None, Some("1970-01-01 00:00:00"), Some("1970-01-01 00:00:05")).map { i =>
+ i.map(Timestamp.valueOf)
+ }
+ val df = values.toDF("c1")
+ val nonNullValues = getNonNullValues[Timestamp](values)
+ val expectedColStatsSeq = df.schema.map { f =>
+ val colStat = ColumnStat(InternalRow(
+ values.count(_.isEmpty).toLong,
+ // Internally, TimestampType is represented as the number of days from 1970-01-01
+ nonNullValues.map(DateTimeUtils.fromJavaTimestamp).max,
+ nonNullValues.map(DateTimeUtils.fromJavaTimestamp).min,
+ nonNullValues.distinct.length.toLong))
+ (f, colStat)
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("column-level statistics for null columns") {
+ val values = Seq(None, None)
+ val data = values.map { i =>
+ (i.map(_.toString), i.map(_.toString.toInt))
+ }
+ val df = data.toDF("c1", "c2")
+ val expectedColStatsSeq = df.schema.map { f =>
+ (f, ColumnStat(InternalRow(values.count(_.isEmpty).toLong, null, null, 0L)))
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("column-level statistics for columns with different types") {
+ val intSeq = Seq(1, 2)
+ val doubleSeq = Seq(1.01d, 2.02d)
+ val stringSeq = Seq("a", "bb")
+ val binarySeq = Seq("a", "bb").map(_.getBytes)
+ val booleanSeq = Seq(true, false)
+ val dateSeq = Seq("1970-01-01", "1970-02-02").map(Date.valueOf)
+ val timestampSeq = Seq("1970-01-01 00:00:00", "1970-01-01 00:00:05").map(Timestamp.valueOf)
+ val longSeq = Seq(5L, 4L)
+
+ val data = intSeq.indices.map { i =>
+ (intSeq(i), doubleSeq(i), stringSeq(i), binarySeq(i), booleanSeq(i), dateSeq(i),
+ timestampSeq(i), longSeq(i))
+ }
+ val df = data.toDF("c1", "c2", "c3", "c4", "c5", "c6", "c7", "c8")
+ val expectedColStatsSeq = df.schema.map { f =>
+ val colStat = f.dataType match {
+ case IntegerType =>
+ ColumnStat(InternalRow(0L, intSeq.max, intSeq.min, intSeq.distinct.length.toLong))
+ case DoubleType =>
+ ColumnStat(InternalRow(0L, doubleSeq.max, doubleSeq.min,
+ doubleSeq.distinct.length.toLong))
+ case StringType =>
+ ColumnStat(InternalRow(0L, stringSeq.map(_.length).sum / stringSeq.length.toDouble,
+ stringSeq.map(_.length).max.toLong, stringSeq.distinct.length.toLong))
+ case BinaryType =>
+ ColumnStat(InternalRow(0L, binarySeq.map(_.length).sum / binarySeq.length.toDouble,
+ binarySeq.map(_.length).max.toLong))
+ case BooleanType =>
+ ColumnStat(InternalRow(0L, booleanSeq.count(_.equals(true)).toLong,
+ booleanSeq.count(_.equals(false)).toLong))
+ case DateType =>
+ ColumnStat(InternalRow(0L, dateSeq.map(DateTimeUtils.fromJavaDate).max,
+ dateSeq.map(DateTimeUtils.fromJavaDate).min, dateSeq.distinct.length.toLong))
+ case TimestampType =>
+ ColumnStat(InternalRow(0L, timestampSeq.map(DateTimeUtils.fromJavaTimestamp).max,
+ timestampSeq.map(DateTimeUtils.fromJavaTimestamp).min,
+ timestampSeq.distinct.length.toLong))
+ case LongType =>
+ ColumnStat(InternalRow(0L, longSeq.max, longSeq.min, longSeq.distinct.length.toLong))
+ }
+ (f, colStat)
+ }
+ checkColStats(df, expectedColStatsSeq)
+ }
+
+ test("update table-level stats while collecting column-level stats") {
+ val table = "tbl"
+ withTable(table) {
+ sql(s"CREATE TABLE $table (c1 int) USING PARQUET")
+ sql(s"INSERT INTO $table SELECT 1")
+ sql(s"ANALYZE TABLE $table COMPUTE STATISTICS")
+ checkTableStats(tableName = table, expectedRowCount = Some(1))
+
+ // update table-level stats between analyze table and analyze column commands
+ sql(s"INSERT INTO $table SELECT 1")
+ sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS c1")
+ val fetchedStats = checkTableStats(tableName = table, expectedRowCount = Some(2))
+
+ val colStat = fetchedStats.get.colStats("c1")
+ StatisticsTest.checkColStat(
+ dataType = IntegerType,
+ colStat = colStat,
+ expectedColStat = ColumnStat(InternalRow(0L, 1, 1, 1L)),
+ rsd = spark.sessionState.conf.ndvMaxError)
+ }
+ }
+
+ test("analyze column stats independently") {
+ val table = "tbl"
+ withTable(table) {
+ sql(s"CREATE TABLE $table (c1 int, c2 long) USING PARQUET")
+ sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS c1")
+ val fetchedStats1 = checkTableStats(tableName = table, expectedRowCount = Some(0))
+ assert(fetchedStats1.get.colStats.size == 1)
+ val expected1 = ColumnStat(InternalRow(0L, null, null, 0L))
+ val rsd = spark.sessionState.conf.ndvMaxError
+ StatisticsTest.checkColStat(
+ dataType = IntegerType,
+ colStat = fetchedStats1.get.colStats("c1"),
+ expectedColStat = expected1,
+ rsd = rsd)
+
+ sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS c2")
+ val fetchedStats2 = checkTableStats(tableName = table, expectedRowCount = Some(0))
+ // column c1 is kept in the stats
+ assert(fetchedStats2.get.colStats.size == 2)
+ StatisticsTest.checkColStat(
+ dataType = IntegerType,
+ colStat = fetchedStats2.get.colStats("c1"),
+ expectedColStat = expected1,
+ rsd = rsd)
+ val expected2 = ColumnStat(InternalRow(0L, null, null, 0L))
+ StatisticsTest.checkColStat(
+ dataType = LongType,
+ colStat = fetchedStats2.get.colStats("c2"),
+ expectedColStat = expected2,
+ rsd = rsd)
+ }
+ }
+}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala
index 264a2ffbeb..8cf42e9248 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala
@@ -18,11 +18,9 @@
package org.apache.spark.sql
import org.apache.spark.sql.catalyst.plans.logical.{GlobalLimit, Join, LocalLimit}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types._
-class StatisticsSuite extends QueryTest with SharedSQLContext {
+class StatisticsSuite extends StatisticsTest {
import testImplicits._
test("SPARK-15392: DataFrame created from RDD should not be broadcasted") {
@@ -77,20 +75,10 @@ class StatisticsSuite extends QueryTest with SharedSQLContext {
}
test("test table-level statistics for data source table created in InMemoryCatalog") {
- def checkTableStats(tableName: String, expectedRowCount: Option[BigInt]): Unit = {
- val df = sql(s"SELECT * FROM $tableName")
- val relations = df.queryExecution.analyzed.collect { case rel: LogicalRelation =>
- assert(rel.catalogTable.isDefined)
- assert(rel.catalogTable.get.stats.flatMap(_.rowCount) === expectedRowCount)
- rel
- }
- assert(relations.size === 1)
- }
-
val tableName = "tbl"
withTable(tableName) {
sql(s"CREATE TABLE $tableName(i INT, j STRING) USING parquet")
- Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.mode("overwrite").insertInto("tbl")
+ Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.mode("overwrite").insertInto(tableName)
// noscan won't count the number of rows
sql(s"ANALYZE TABLE $tableName COMPUTE STATISTICS noscan")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsTest.scala
new file mode 100644
index 0000000000..5134ac0e7e
--- /dev/null
+++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsTest.scala
@@ -0,0 +1,129 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Statistics}
+import org.apache.spark.sql.execution.command.{AnalyzeColumnCommand, ColumnStatStruct}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.test.SharedSQLContext
+import org.apache.spark.sql.types._
+
+trait StatisticsTest extends QueryTest with SharedSQLContext {
+
+ def checkColStats(
+ df: DataFrame,
+ expectedColStatsSeq: Seq[(StructField, ColumnStat)]): Unit = {
+ val table = "tbl"
+ withTable(table) {
+ df.write.format("json").saveAsTable(table)
+ val columns = expectedColStatsSeq.map(_._1)
+ val tableIdent = TableIdentifier(table, Some("default"))
+ val relation = spark.sessionState.catalog.lookupRelation(tableIdent)
+ val (_, columnStats) =
+ AnalyzeColumnCommand(tableIdent, columns.map(_.name)).computeColStats(spark, relation)
+ expectedColStatsSeq.foreach { case (field, expectedColStat) =>
+ assert(columnStats.contains(field.name))
+ val colStat = columnStats(field.name)
+ StatisticsTest.checkColStat(
+ dataType = field.dataType,
+ colStat = colStat,
+ expectedColStat = expectedColStat,
+ rsd = spark.sessionState.conf.ndvMaxError)
+
+ // check if we get the same colStat after encoding and decoding
+ val encodedCS = colStat.toString
+ val numFields = ColumnStatStruct.numStatFields(field.dataType)
+ val decodedCS = ColumnStat(numFields, encodedCS)
+ StatisticsTest.checkColStat(
+ dataType = field.dataType,
+ colStat = decodedCS,
+ expectedColStat = expectedColStat,
+ rsd = spark.sessionState.conf.ndvMaxError)
+ }
+ }
+ }
+
+ def checkTableStats(tableName: String, expectedRowCount: Option[Int]): Option[Statistics] = {
+ val df = spark.table(tableName)
+ val stats = df.queryExecution.analyzed.collect { case rel: LogicalRelation =>
+ assert(rel.catalogTable.get.stats.flatMap(_.rowCount) === expectedRowCount)
+ rel.catalogTable.get.stats
+ }
+ assert(stats.size == 1)
+ stats.head
+ }
+}
+
+object StatisticsTest {
+ def checkColStat(
+ dataType: DataType,
+ colStat: ColumnStat,
+ expectedColStat: ColumnStat,
+ rsd: Double): Unit = {
+ dataType match {
+ case StringType =>
+ val cs = colStat.forString
+ val expectedCS = expectedColStat.forString
+ assert(cs.numNulls == expectedCS.numNulls)
+ assert(cs.avgColLen == expectedCS.avgColLen)
+ assert(cs.maxColLen == expectedCS.maxColLen)
+ checkNdv(ndv = cs.ndv, expectedNdv = expectedCS.ndv, rsd = rsd)
+ case BinaryType =>
+ val cs = colStat.forBinary
+ val expectedCS = expectedColStat.forBinary
+ assert(cs.numNulls == expectedCS.numNulls)
+ assert(cs.avgColLen == expectedCS.avgColLen)
+ assert(cs.maxColLen == expectedCS.maxColLen)
+ case BooleanType =>
+ val cs = colStat.forBoolean
+ val expectedCS = expectedColStat.forBoolean
+ assert(cs.numNulls == expectedCS.numNulls)
+ assert(cs.numTrues == expectedCS.numTrues)
+ assert(cs.numFalses == expectedCS.numFalses)
+ case atomicType: AtomicType =>
+ checkNumericColStats(
+ dataType = atomicType, colStat = colStat, expectedColStat = expectedColStat, rsd = rsd)
+ }
+ }
+
+ private def checkNumericColStats(
+ dataType: AtomicType,
+ colStat: ColumnStat,
+ expectedColStat: ColumnStat,
+ rsd: Double): Unit = {
+ val cs = colStat.forNumeric(dataType)
+ val expectedCS = expectedColStat.forNumeric(dataType)
+ assert(cs.numNulls == expectedCS.numNulls)
+ assert(cs.max == expectedCS.max)
+ assert(cs.min == expectedCS.min)
+ checkNdv(ndv = cs.ndv, expectedNdv = expectedCS.ndv, rsd = rsd)
+ }
+
+ private def checkNdv(ndv: Long, expectedNdv: Long, rsd: Double): Unit = {
+ // ndv is an approximate value, so we make sure we have the value, and it should be
+ // within 3*SD's of the given rsd.
+ if (expectedNdv == 0) {
+ assert(ndv == 0)
+ } else if (expectedNdv > 0) {
+ assert(ndv > 0)
+ val error = math.abs((ndv / expectedNdv.toDouble) - 1.0d)
+ assert(error <= rsd * 3.0d, "Error should be within 3 std. errors.")
+ }
+ }
+}