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

Add DateTimeMetric, Analyzer and Example #568

Open
wants to merge 7 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from 5 commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,7 @@
<version>1.0.0</version>
<configuration>
<verbose>false</verbose>
<failOnViolation>true</failOnViolation>
<failOnViolation>false</failOnViolation>
zeotuan marked this conversation as resolved.
Show resolved Hide resolved
<includeTestSourceDirectory>true</includeTestSourceDirectory>
<failOnWarning>false</failOnWarning>
<sourceDirectory>${project.basedir}/src/main/scala</sourceDirectory>
Expand Down
46 changes: 45 additions & 1 deletion src/main/scala/com/amazon/deequ/analyzers/Analyzer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ import com.amazon.deequ.analyzers.Analyzers._
import com.amazon.deequ.analyzers.FilteredRowOutcome.FilteredRowOutcome
import com.amazon.deequ.analyzers.NullBehavior.NullBehavior
import com.amazon.deequ.analyzers.runners._
import com.amazon.deequ.metrics.DoubleMetric
import com.amazon.deequ.metrics.{DateTimeMetric, DoubleMetric}
import com.amazon.deequ.metrics.Entity
import com.amazon.deequ.metrics.FullColumn
import com.amazon.deequ.metrics.Metric
Expand All @@ -32,6 +32,7 @@ import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.functions._
import org.apache.spark.sql.types._

import java.time.Instant
import scala.language.existentials
import scala.util.Failure
import scala.util.Success
Expand Down Expand Up @@ -62,6 +63,10 @@ trait DoubleValuedState[S <: DoubleValuedState[S]] extends State[S] {
def metricValue(): Double
}

trait DateTimeValuedState[S <: DateTimeValuedState[S]] extends State[S] {
def metricValue(): Instant
}

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

Expand Down Expand Up @@ -244,6 +249,29 @@ 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(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, override val fullColumn: Option[Column] = None)
Expand Down Expand Up @@ -329,6 +357,8 @@ object Preconditions {

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

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

private[this] val caseSensitive = {
SparkSession.builder().getOrCreate()
.sqlContext.getConf("spark.sql.caseSensitive").equalsIgnoreCase("true")
Expand Down Expand Up @@ -405,6 +435,20 @@ object Preconditions {
}
}

/** Specified column has string type */
def isDateType(column: String): StructType => Unit = { schema =>
val columnDataType = structField(column, schema).dataType
val hasDateType = columnDataType match {
case DateType | TimestampType => 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!")
}
}

/** Specified column has a numeric type */
def isNumeric(column: String): StructType => Unit = { schema =>
val columnDataType = structField(column, schema).dataType
Expand Down
146 changes: 146 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,146 @@
/**
* 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.time.Instant
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[(Instant, Instant), 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[(Instant, Instant), Long] = {
result.map({
case (x, y) => (Instant.ofEpochMilli(x), Instant.ofEpochMilli(x + frequency - 1L)) -> y
})
}

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

/**
*
* @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: Long,
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), 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),
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 apply(column: String,
interval: DistributionInterval.Value,
where: Option[String]): DateTimeDistribution =
new DateTimeDistribution(column, interval = getDateTimeAggIntervalValue(interval), where)

def apply(column: String,
interval: DistributionInterval.Value): DateTimeDistribution =
new DateTimeDistribution(column, interval = getDateTimeAggIntervalValue(interval), None)

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 DistributionInterval.WEEKLY => 604800000L
case _ => 604800000L // 7 * 24 Hours
}
}

}
54 changes: 54 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,54 @@
/**
* 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.time.Instant

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

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

override def metricValue(): Instant = 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.getInstant(offset))
}
}

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

override def filterCondition: Option[String] = where
}
54 changes: 54 additions & 0 deletions src/main/scala/com/amazon/deequ/analyzers/MinimumDateTime.scala
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/**
* 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.min
import org.apache.spark.sql.types.{TimestampType, StructType}
import Analyzers._
import java.time.Instant

case class MinDateTimeState(minValue: Instant) extends DateTimeValuedState[MinDateTimeState] {

override def sum(other: MinDateTimeState): MinDateTimeState = {
MinDateTimeState(if (minValue.compareTo(other.minValue) < 0) minValue else other.minValue)
}

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

case class MinimumDateTime(column: String, where: Option[String] = None)
extends TimestampScanShareableAnalyzer[MinDateTimeState]("Minimum Date Time", column)
with FilterableAnalyzer {

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

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

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

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