Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Features and enhancements done #299

Open
wants to merge 14 commits into
base: master
Choose a base branch
from
109 changes: 108 additions & 1 deletion src/main/scala/com/amazon/deequ/analyzers/Analyzer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -16,8 +16,10 @@

package com.amazon.deequ.analyzers

import java.math.BigDecimal

import com.amazon.deequ.analyzers.Analyzers._
import com.amazon.deequ.metrics.{DoubleMetric, Entity, Metric}
import com.amazon.deequ.metrics.{BigDecimalMetric, DoubleMetric, Entity, Metric, DateTimeMetric}
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._
import org.apache.spark.sql.{Column, DataFrame, Row, SparkSession}
Expand Down Expand Up @@ -52,6 +54,11 @@ trait DoubleValuedState[S <: DoubleValuedState[S]] extends State[S] {
def metricValue(): Double
}

/** A state which produces a BigDecimalValued metric */
trait BigDecimalValuedState[S <: BigDecimalValuedState[S]] extends State[S] {
def metricValue(): BigDecimal
}

/** Common trait for all analyzers which generates metrics from states computed on data frames */
trait Analyzer[S <: State[_], +M <: Metric[_]] {

Expand Down Expand Up @@ -225,6 +232,68 @@ abstract class StandardScanShareableAnalyzer[S <: DoubleValuedState[_]](
}
}

/** A scan-shareable analyzer that produces a DateTimeMetric */
abstract class TimestampScanShareableAnalyzer[S <: DateTimeValuedState[_]](
name: String,
instance: String,
entity: Entity.Value = Entity.Column)
extends ScanShareableAnalyzer[S, DateTimeMetric] {

override def computeMetricFrom(state: Option[S]): DateTimeMetric = {
state match {
case Some(theState) =>
DateTimeMetric(entity, name, instance, Success(theState.metricValue()))
case _ =>
DateTimeMetric(entity, name, instance, Failure(
MetricCalculationException.wrapIfNecessary(emptyStateException(this))))
}
}

override private[deequ] def toFailureMetric(exception: Exception): DateTimeMetric = {
DateTimeMetric(entity, name, instance, Failure(
MetricCalculationException.wrapIfNecessary(exception)))
}

override def preconditions: Seq[StructType => Unit] = {
additionalPreconditions() ++ super.preconditions
}

protected def additionalPreconditions(): Seq[StructType => Unit] = {
Seq.empty
}
}

/** A scan-shareable analyzer that produces a BigDecimalMetric */
abstract class BigDecimalScanShareableAnalyzer[S <: BigDecimalValuedState[_]](
name: String,
instance: String,
entity: Entity.Value = Entity.Column)
extends ScanShareableAnalyzer[S, BigDecimalMetric] {

override def computeMetricFrom(state: Option[S]): BigDecimalMetric = {
state match {
case Some(theState) =>
BigDecimalMetric(entity, name, instance, Success(theState.metricValue()))
case _ =>
BigDecimalMetric(entity, name, instance, Failure(
MetricCalculationException.wrapIfNecessary(emptyStateException(this))))
}
}

override private[deequ] def toFailureMetric(exception: Exception): BigDecimalMetric = {
BigDecimalMetric(entity, name, instance, Failure(
MetricCalculationException.wrapIfNecessary(exception)))
}

override def preconditions: Seq[StructType => Unit] = {
additionalPreconditions() ++ super.preconditions
}

protected def additionalPreconditions(): Seq[StructType => Unit] = {
Seq.empty
}
}

/** A state for computing ratio-based metrics,
* contains #rows that match a predicate and overall #rows */
case class NumMatchesAndCount(numMatches: Long, count: Long)
Expand Down Expand Up @@ -287,6 +356,9 @@ object Preconditions {
private[this] val numericDataTypes =
Set(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, DecimalType)

private[this] val dateTypes =
Set(TimestampType, DateType)

private[this] val nestedDataTypes = Set(StructType, MapType, ArrayType)

private[this] val caseSensitive = {
Expand All @@ -304,6 +376,8 @@ object Preconditions {
}
}



def hasColumn(column: String, schema: StructType): Boolean = {
if (caseSensitive) {
schema.fieldNames.contains(column)
Expand Down Expand Up @@ -380,6 +454,39 @@ object Preconditions {
}
}

/** Asserts if Specified column is a DateType or TimestampType type throw Exception if not
* @param column for which assertion is performed
* @return
* */
def isDateType(column: String): StructType => Unit = { schema =>
val columnDataType = structField(column, schema).dataType
val hasDateType = columnDataType match {
case TimestampType | DateType => true
case _ => false
}
if (!hasDateType) {
throw new WrongColumnTypeException(s"Expected type of column $column to be one of " +
s"(${dateTypes.mkString(",")}), but found $columnDataType instead!")
}
}

/** Asserts if Specified column is a Decimal type throw Exception if not
* @param column for which assertion is performed
* @return
* */
def isDecimalType(column: String): StructType => Unit = { schema =>
val columnDataType = structField(column, schema).dataType
val hasNumericType = columnDataType match {
case _ : DecimalType => true
case _ => false
}

if (!hasNumericType) {
throw new WrongColumnTypeException(s"Expected type of column $column to be one of " +
s"(${numericDataTypes.mkString(",")}), but found $columnDataType instead!")
}
}

/** Specified column has string type */
def isString(column: String): StructType => Unit = { schema =>
val columnDataType = structField(column, schema).dataType
Expand Down
129 changes: 129 additions & 0 deletions src/main/scala/com/amazon/deequ/analyzers/DateTimeDistribution.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,129 @@
/**
* Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
*
* Licensed under the Apache License, Version 2.0 (the "License"). You may not
* use this file except in compliance with the License. A copy of the License
* is located at
*
* http://aws.amazon.com/apache2.0/
*
* or in the "license" file accompanying this file. This file 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 com.amazon.deequ.analyzers

import java.sql.Timestamp
import com.amazon.deequ.analyzers.Analyzers._
import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isDateType}
import com.amazon.deequ.analyzers.runners.MetricCalculationException
import com.amazon.deequ.metrics.{Distribution, DistributionValue, HistogramMetric}
import org.apache.spark.sql.DeequFunctions.dateTimeDistribution
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{Column, Row}

import scala.util.{Failure, Success}

object DistributionInterval extends Enumeration {
val QUARTER_HOUR, HOURLY, DAILY, WEEKLY, MONTHLY = Value
}

case class DateTimeDistributionState(distribution: Map[(Timestamp, Timestamp), Long])
extends State[DateTimeDistributionState] {

override def sum(other: DateTimeDistributionState): DateTimeDistributionState = {

DateTimeDistributionState(distribution ++ other.distribution.map {
case (k, v) => k -> (v + distribution.getOrElse(k, 0L))
})
}
}

object DateTimeDistributionState {

def computeStateFromResult(result: Map[Long, Long], frequency: Long):
Map[(Timestamp, Timestamp), Long] = {
result.map({
case (x, y) => (new Timestamp(x), new Timestamp(x + frequency - 1L)) -> y
})
}

def toDistribution(histogram: DateTimeDistributionState): Distribution = {
val totalCount = histogram.distribution.foldLeft(0L)(_ + _._2)
Distribution(histogram.distribution.map({
case (x, y) => ("(" + x._1.toString + " to " + x._2.toString + ")") ->
DistributionValue(y, y.toDouble / totalCount)
}), totalCount)
}
}

/**
*
* @param column : column on which distribution analysis is to be performed
* @param interval : interval of the distribution;
* @param where : optional filter condition
*/
case class DateTimeDistribution(
column: String,
interval: DistributionInterval.Value,
where: Option[String] = None)
extends ScanShareableAnalyzer[DateTimeDistributionState, HistogramMetric]
with FilterableAnalyzer {

/** Defines the aggregations to compute on the data */
override private[deequ] def aggregationFunctions(): Seq[Column] = {
dateTimeDistribution(conditionalSelection(column, where),
DateTimeDistribution.getDateTimeAggIntervalValue(interval)) :: Nil
}

/** Computes the state from the result of the aggregation functions */
override private[deequ] def fromAggregationResult(result: Row, offset: Int):
Option[DateTimeDistributionState] = {
ifNoNullsIn(result, offset) { _ =>
DateTimeDistributionState(
DateTimeDistributionState.computeStateFromResult(Map.empty[Long, Long] ++ result.getMap(0),
DateTimeDistribution.getDateTimeAggIntervalValue(interval)))
}
}

override def preconditions: Seq[StructType => Unit] = {
hasColumn(column) +: isDateType(column) +: super.preconditions
}

override def filterCondition: Option[String] = where

/**
* Compute the metric from the state (sufficient statistics)
*
* @param state wrapper holding a state of type S (required due to typing issues...)
* @return
*/
override def computeMetricFrom(state: Option[DateTimeDistributionState]): HistogramMetric = {
state match {
case Some(histogram) =>
HistogramMetric(column, Success(DateTimeDistributionState.toDistribution(histogram)))
case _ =>
toFailureMetric(emptyStateException(this))
}
}

override private[deequ] def toFailureMetric(failure: Exception): HistogramMetric = {
HistogramMetric(column, Failure(MetricCalculationException.wrapIfNecessary(failure)))
}
}

object DateTimeDistribution {

def getDateTimeAggIntervalValue(interval: DistributionInterval.Value): Long = {
interval match {
case DistributionInterval.QUARTER_HOUR => 900000L // 15 Minutes
case DistributionInterval.HOURLY => 3600000L // 60 Minutes
case DistributionInterval.DAILY => 86400000L // 24 Hours
case _ => 604800000L // 7 * 24 Hours
}
}

}
40 changes: 38 additions & 2 deletions src/main/scala/com/amazon/deequ/analyzers/Maximum.scala
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,11 @@

package com.amazon.deequ.analyzers

import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isNumeric}
import java.math.BigDecimal

import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isDecimalType, isNumeric}
import org.apache.spark.sql.{Column, Row}
import org.apache.spark.sql.functions.max
import org.apache.spark.sql.functions.{max, min}
import org.apache.spark.sql.types.{DoubleType, StructType}
import Analyzers._

Expand Down Expand Up @@ -54,3 +56,37 @@ case class Maximum(column: String, where: Option[String] = None)

override def filterCondition: Option[String] = where
}

case class MaxBigDecimalState(minValue: BigDecimal)
extends BigDecimalValuedState[MaxBigDecimalState] {

override def sum(other: MaxBigDecimalState): MaxBigDecimalState = {
MaxBigDecimalState(minValue.max(other.minValue))
}

override def metricValue(): BigDecimal = {
minValue
}
}

case class MaximumBigDecimal(column: String, where: Option[String] = None)
extends BigDecimalScanShareableAnalyzer[MaxBigDecimalState]("Maximum BigDecimal", column)
with FilterableAnalyzer {

override def aggregationFunctions(): Seq[Column] = {
max(conditionalSelection(column, where)) :: Nil
}


override def fromAggregationResult(result: Row, offset: Int): Option[MaxBigDecimalState] = {
ifNoNullsIn(result, offset) { _ =>
MaxBigDecimalState(result.getDecimal(offset))
}
}

override protected def additionalPreconditions(): Seq[StructType => Unit] = {
hasColumn(column) :: isDecimalType(column) :: Nil
}

override def filterCondition: Option[String] = where
}
56 changes: 56 additions & 0 deletions src/main/scala/com/amazon/deequ/analyzers/MaximumDateTime.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
/**
* Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
*
* Licensed under the Apache License, Version 2.0 (the "License"). You may not
* use this file except in compliance with the License. A copy of the License
* is located at
*
* http://aws.amazon.com/apache2.0/
*
* or in the "license" file accompanying this file. This file 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 com.amazon.deequ.analyzers

import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isDateType}
import org.apache.spark.sql.{Column, Row}
import org.apache.spark.sql.functions.max
import org.apache.spark.sql.types.{TimestampType, StructType}
import Analyzers._
import java.sql.Timestamp

case class MaxDateTimeState(maxValue: Timestamp) extends DateTimeValuedState[MaxDateTimeState] {

override def sum(other: MaxDateTimeState): MaxDateTimeState = {
MaxDateTimeState(if (maxValue.compareTo(other.maxValue) > 0) maxValue else other.maxValue)
}

override def metricValue(): Timestamp = {
maxValue
}
}

case class MaximumDateTime(column: String, where: Option[String] = None)
extends TimestampScanShareableAnalyzer[MaxDateTimeState]("Maximum Date Time", column)
with FilterableAnalyzer {

override def aggregationFunctions(): Seq[Column] = {
max(conditionalSelection(column, where)).cast(TimestampType) :: Nil
}

override def fromAggregationResult(result: Row, offset: Int): Option[MaxDateTimeState] = {
ifNoNullsIn(result, offset) { _ =>
MaxDateTimeState(result.getTimestamp(offset))
}
}

override protected def additionalPreconditions(): Seq[StructType => Unit] = {
hasColumn(column) :: isDateType(column) :: Nil
}

override def filterCondition: Option[String] = where
}
Loading