/
StatFunctions.scala
304 lines (279 loc) · 13.4 KB
/
StatFunctions.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
/*
* 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.execution.stat
import java.util.Locale
import org.apache.spark.internal.Logging
import org.apache.spark.sql.{Column, DataFrame, Dataset, Row}
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Cast, Expression, GenericInternalRow, GetArrayItem, Literal}
import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
import org.apache.spark.sql.catalyst.util.{GenericArrayData, QuantileSummaries}
import org.apache.spark.sql.errors.QueryExecutionErrors
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
object StatFunctions extends Logging {
/**
* Calculates the approximate quantiles of multiple numerical columns of a DataFrame in one pass.
*
* The result of this algorithm has the following deterministic bound:
* If the DataFrame has N elements and if we request the quantile at probability `p` up to error
* `err`, then the algorithm will return a sample `x` from the DataFrame so that the *exact* rank
* of `x` is close to (p * N).
* More precisely,
*
* floor((p - err) * N) <= rank(x) <= ceil((p + err) * N).
*
* This method implements a variation of the Greenwald-Khanna algorithm (with some speed
* optimizations).
* The algorithm was first present in <a href="https://doi.org/10.1145/375663.375670">
* Space-efficient Online Computation of Quantile Summaries</a> by Greenwald and Khanna.
*
* @param df the dataframe
* @param cols numerical columns of the dataframe
* @param probabilities a list of quantile probabilities
* Each number must belong to [0, 1].
* For example 0 is the minimum, 0.5 is the median, 1 is the maximum.
* @param relativeError The relative target precision to achieve (greater than or equal 0).
* If set to zero, the exact quantiles are computed, which could be very expensive.
* Note that values greater than 1 are accepted but give the same result as 1.
*
* @return for each column, returns the requested approximations
*
* @note null and NaN values will be ignored in numerical columns before calculation. For
* a column only containing null or NaN values, an empty array is returned.
*/
def multipleApproxQuantiles(
df: DataFrame,
cols: Seq[String],
probabilities: Seq[Double],
relativeError: Double): Seq[Seq[Double]] = {
require(relativeError >= 0,
s"Relative Error must be non-negative but got $relativeError")
val columns: Seq[Column] = cols.map { colName =>
val field = df.resolve(colName)
require(field.dataType.isInstanceOf[NumericType],
s"Quantile calculation for column $colName with data type ${field.dataType}" +
" is not supported.")
Column(Cast(Column(colName).expr, DoubleType))
}
val emptySummaries = Array.fill(cols.size)(
new QuantileSummaries(QuantileSummaries.defaultCompressThreshold, relativeError))
// Note that it works more or less by accident as `rdd.aggregate` is not a pure function:
// this function returns the same array as given in the input (because `aggregate` reuses
// the same argument).
def apply(summaries: Array[QuantileSummaries], row: Row): Array[QuantileSummaries] = {
var i = 0
while (i < summaries.length) {
if (!row.isNullAt(i)) {
val v = row.getDouble(i)
if (!v.isNaN) summaries(i) = summaries(i).insert(v)
}
i += 1
}
summaries
}
def merge(
sum1: Array[QuantileSummaries],
sum2: Array[QuantileSummaries]): Array[QuantileSummaries] = {
sum1.zip(sum2).map { case (s1, s2) => s1.compress().merge(s2.compress()) }
}
val summaries = df.select(columns: _*).rdd.treeAggregate(emptySummaries)(apply, merge)
summaries.map { summary => probabilities.flatMap(summary.query) }
}
/** Calculate the Pearson Correlation Coefficient for the given columns */
def pearsonCorrelation(df: DataFrame, cols: Seq[String]): Double = {
val counts = collectStatisticalData(df, cols, "correlation")
counts.Ck / math.sqrt(counts.MkX * counts.MkY)
}
/** Helper class to simplify tracking and merging counts. */
private class CovarianceCounter extends Serializable {
var xAvg = 0.0 // the mean of all examples seen so far in col1
var yAvg = 0.0 // the mean of all examples seen so far in col2
var Ck = 0.0 // the co-moment after k examples
var MkX = 0.0 // sum of squares of differences from the (current) mean for col1
var MkY = 0.0 // sum of squares of differences from the (current) mean for col2
var count = 0L // count of observed examples
// add an example to the calculation
def add(x: Double, y: Double): this.type = {
val deltaX = x - xAvg
val deltaY = y - yAvg
count += 1
xAvg += deltaX / count
yAvg += deltaY / count
Ck += deltaX * (y - yAvg)
MkX += deltaX * (x - xAvg)
MkY += deltaY * (y - yAvg)
this
}
// merge counters from other partitions. Formula can be found at:
// http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance
def merge(other: CovarianceCounter): this.type = {
if (other.count > 0) {
val totalCount = count + other.count
val deltaX = xAvg - other.xAvg
val deltaY = yAvg - other.yAvg
Ck += other.Ck + deltaX * deltaY * count / totalCount * other.count
xAvg = (xAvg * count + other.xAvg * other.count) / totalCount
yAvg = (yAvg * count + other.yAvg * other.count) / totalCount
MkX += other.MkX + deltaX * deltaX * count / totalCount * other.count
MkY += other.MkY + deltaY * deltaY * count / totalCount * other.count
count = totalCount
}
this
}
// return the sample covariance for the observed examples
def cov: Double = Ck / (count - 1)
}
private def collectStatisticalData(df: DataFrame, cols: Seq[String],
functionName: String): CovarianceCounter = {
require(cols.length == 2, s"Currently $functionName calculation is supported " +
"between two columns.")
cols.map(name => (name, df.resolve(name))).foreach { case (name, data) =>
require(data.dataType.isInstanceOf[NumericType], s"Currently $functionName calculation " +
s"for columns with dataType ${data.dataType.catalogString} not supported.")
}
val columns = cols.map(n => Column(Cast(Column(n).expr, DoubleType)))
df.select(columns: _*).queryExecution.toRdd.treeAggregate(new CovarianceCounter)(
seqOp = (counter, row) => {
counter.add(row.getDouble(0), row.getDouble(1))
},
combOp = (baseCounter, other) => {
baseCounter.merge(other)
})
}
/**
* Calculate the covariance of two numerical columns of a DataFrame.
* @param df The DataFrame
* @param cols the column names
* @return the covariance of the two columns.
*/
def calculateCov(df: DataFrame, cols: Seq[String]): Double = {
val counts = collectStatisticalData(df, cols, "covariance")
counts.cov
}
/** Generate a table of frequencies for the elements of two columns. */
def crossTabulate(df: DataFrame, col1: String, col2: String): DataFrame = {
val tableName = s"${col1}_$col2"
val counts = df.groupBy(col1, col2).agg(count("*")).take(1e6.toInt)
if (counts.length == 1e6.toInt) {
logWarning("The maximum limit of 1e6 pairs have been collected, which may not be all of " +
"the pairs. Please try reducing the amount of distinct items in your columns.")
}
def cleanElement(element: Any): String = {
if (element == null) "null" else element.toString
}
// get the distinct sorted values of column 2, so that we can make them the column names
val distinctCol2: Map[Any, Int] =
counts.map(e => cleanElement(e.get(1))).distinct.sorted.zipWithIndex.toMap
val columnSize = distinctCol2.size
require(columnSize < 1e4, s"The number of distinct values for $col2, can't " +
s"exceed 1e4. Currently $columnSize")
val table = counts.groupBy(_.get(0)).map { case (col1Item, rows) =>
val countsRow = new GenericInternalRow(columnSize + 1)
rows.foreach { (row: Row) =>
// row.get(0) is column 1
// row.get(1) is column 2
// row.get(2) is the frequency
val columnIndex = distinctCol2(cleanElement(row.get(1)))
countsRow.setLong(columnIndex + 1, row.getLong(2))
}
// the value of col1 is the first value, the rest are the counts
countsRow.update(0, UTF8String.fromString(cleanElement(col1Item)))
countsRow
}.toSeq
// Back ticks can't exist in DataFrame column names, therefore drop them. To be able to accept
// special keywords and `.`, wrap the column names in ``.
def cleanColumnName(name: String): String = {
name.replace("`", "")
}
// In the map, the column names (._1) are not ordered by the index (._2). This was the bug in
// SPARK-8681. We need to explicitly sort by the column index and assign the column names.
val headerNames = distinctCol2.toSeq.sortBy(_._2).map { r =>
StructField(cleanColumnName(r._1.toString), LongType)
}
val schema = StructType(StructField(tableName, StringType) +: headerNames)
Dataset.ofRows(df.sparkSession, LocalRelation(schema.toAttributes, table)).na.fill(0.0)
}
/** Calculate selected summary statistics for a dataset */
def summary(ds: Dataset[_], statistics: Seq[String]): DataFrame = {
val defaultStatistics = Seq("count", "mean", "stddev", "min", "25%", "50%", "75%", "max")
val selectedStatistics = if (statistics.nonEmpty) statistics else defaultStatistics
val percentiles = selectedStatistics.filter(a => a.endsWith("%")).map { p =>
try {
p.stripSuffix("%").toDouble / 100.0
} catch {
case e: NumberFormatException =>
throw QueryExecutionErrors.cannotParseStatisticAsPercentileError(p, e)
}
}
require(percentiles.forall(p => p >= 0 && p <= 1), "Percentiles must be in the range [0, 1]")
var percentileIndex = 0
val statisticFns = selectedStatistics.map { stats =>
if (stats.endsWith("%")) {
val index = percentileIndex
percentileIndex += 1
(child: Expression) =>
GetArrayItem(
new ApproximatePercentile(child,
Literal(new GenericArrayData(percentiles), ArrayType(DoubleType, false)))
.toAggregateExpression(),
Literal(index))
} else {
stats.toLowerCase(Locale.ROOT) match {
case "count" => (child: Expression) => Count(child).toAggregateExpression()
case "count_distinct" => (child: Expression) =>
Count(child).toAggregateExpression(isDistinct = true)
case "approx_count_distinct" => (child: Expression) =>
HyperLogLogPlusPlus(child).toAggregateExpression()
case "mean" => (child: Expression) => Average(child).toAggregateExpression()
case "stddev" => (child: Expression) => StddevSamp(child).toAggregateExpression()
case "min" => (child: Expression) => Min(child).toAggregateExpression()
case "max" => (child: Expression) => Max(child).toAggregateExpression()
case _ => throw QueryExecutionErrors.statisticNotRecognizedError(stats)
}
}
}
val selectedCols = ds.logicalPlan.output
.filter(a => a.dataType.isInstanceOf[NumericType] || a.dataType.isInstanceOf[StringType])
val aggExprs = statisticFns.flatMap { func =>
selectedCols.map(c => Column(Cast(func(c), StringType)).as(c.name))
}
// If there is no selected columns, we don't need to run this aggregate, so make it a lazy val.
lazy val aggResult = ds.select(aggExprs: _*).queryExecution.toRdd.collect().head
// We will have one row for each selected statistic in the result.
val result = Array.fill[InternalRow](selectedStatistics.length) {
// each row has the statistic name, and statistic values of each selected column.
new GenericInternalRow(selectedCols.length + 1)
}
var rowIndex = 0
while (rowIndex < result.length) {
val statsName = selectedStatistics(rowIndex)
result(rowIndex).update(0, UTF8String.fromString(statsName))
for (colIndex <- selectedCols.indices) {
val statsValue = aggResult.getUTF8String(rowIndex * selectedCols.length + colIndex)
result(rowIndex).update(colIndex + 1, statsValue)
}
rowIndex += 1
}
// All columns are string type
val output = AttributeReference("summary", StringType)() +:
selectedCols.map(c => AttributeReference(c.name, StringType)())
Dataset.ofRows(ds.sparkSession, LocalRelation(output, result))
}
}