Skip to content

Commit 7c76c59

Browse files
eycho-ammarcantonyshriyavanvarisvanvarimentekid
authored
Add commits from master branch to release/2.0.8-spark-3.5 (#587)
* Replace 'withColumns' with 'select' (#582) 'withColumns' was introduced in Spark 3.3, so it won't work for Deequ's <3.3 builds. * Replace rdd with dataframe functions in Histogram analyzer (#586) Co-authored-by: Shriya Vanvari <svanvari@amazon.com> * Updated version in pom.xml to 2.0.8-spark-3.5 (#578) Co-authored-by: Yannis Mentekidis <mentekid@amazon.com> --------- Co-authored-by: Josh <5685731+marcantony@users.noreply.github.com> Co-authored-by: Shriya Vanvari <vanvari.shriya@gmail.com> Co-authored-by: Shriya Vanvari <svanvari@amazon.com> Co-authored-by: Yannis Mentekidis <mentekid@users.noreply.github.com> Co-authored-by: Yannis Mentekidis <mentekid@amazon.com>
1 parent ff3cb69 commit 7c76c59

File tree

2 files changed

+21
-5
lines changed

2 files changed

+21
-5
lines changed

src/main/scala/com/amazon/deequ/VerificationResult.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -31,7 +31,7 @@ import com.amazon.deequ.repository.SimpleResultSerde
3131
import org.apache.spark.sql.Column
3232
import org.apache.spark.sql.DataFrame
3333
import org.apache.spark.sql.SparkSession
34-
import org.apache.spark.sql.functions.monotonically_increasing_id
34+
import org.apache.spark.sql.functions.{col, monotonically_increasing_id}
3535

3636
import java.util.UUID
3737

@@ -96,9 +96,10 @@ object VerificationResult {
9696
data: DataFrame): DataFrame = {
9797

9898
val columnNamesToMetrics: Map[String, Column] = verificationResultToColumn(verificationResult)
99+
val columnsAliased = columnNamesToMetrics.toSeq.map { case (name, col) => col.as(name) }
99100

100101
val dataWithID = data.withColumn(UNIQUENESS_ID, monotonically_increasing_id())
101-
dataWithID.withColumns(columnNamesToMetrics).drop(UNIQUENESS_ID)
102+
dataWithID.select(col("*") +: columnsAliased: _*).drop(UNIQUENESS_ID)
102103
}
103104

104105
def checkResultsAsJson(verificationResult: VerificationResult,

src/main/scala/com/amazon/deequ/analyzers/Histogram.scala

Lines changed: 18 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -84,11 +84,26 @@ case class Histogram(
8484
case Some(theState) =>
8585
val value: Try[Distribution] = Try {
8686

87-
val topNRows = theState.frequencies.rdd.top(maxDetailBins)(OrderByAbsoluteCount)
87+
val countColumnName = theState.frequencies.schema.fields
88+
.find(field => field.dataType == LongType && field.name != column)
89+
.map(_.name)
90+
.getOrElse(throw new IllegalStateException(s"Count column not found in the frequencies DataFrame"))
91+
92+
val topNRowsDF = theState.frequencies
93+
.orderBy(col(countColumnName).desc)
94+
.limit(maxDetailBins)
95+
.collect()
96+
8897
val binCount = theState.frequencies.count()
8998

90-
val histogramDetails = topNRows
91-
.map { case Row(discreteValue: String, absolute: Long) =>
99+
val columnName = theState.frequencies.columns
100+
.find(_ == column)
101+
.getOrElse(throw new IllegalStateException(s"Column $column not found"))
102+
103+
val histogramDetails = topNRowsDF
104+
.map { row =>
105+
val discreteValue = row.getAs[String](columnName)
106+
val absolute = row.getAs[Long](countColumnName)
92107
val ratio = absolute.toDouble / theState.numRows
93108
discreteValue -> DistributionValue(absolute, ratio)
94109
}

0 commit comments

Comments
 (0)