├── .github
├── ISSUE_TEMPLATE
│ ├── bug_report.md
│ ├── feature_request.md
│ └── general-inquiry.md
├── PULL_REQUEST_TEMPLATE.md
└── workflows
│ └── maven.yml
├── .gitignore
├── .travis.yml
├── CODE_OF_CONDUCT.md
├── CONTRIBUTING.md
├── LICENSE
├── Makefile
├── NOTICE
├── README.md
├── deequ-scalastyle.xml
├── docs
└── key-concepts.md
├── pom.xml
├── settings.xml
├── src
├── main
│ └── scala
│ │ └── com
│ │ └── amazon
│ │ └── deequ
│ │ ├── VerificationResult.scala
│ │ ├── VerificationRunBuilder.scala
│ │ ├── VerificationSuite.scala
│ │ ├── analyzers
│ │ ├── Analysis.scala
│ │ ├── Analyzer.scala
│ │ ├── ApproxCountDistinct.scala
│ │ ├── ApproxQuantile.scala
│ │ ├── ApproxQuantiles.scala
│ │ ├── ColumnCount.scala
│ │ ├── Completeness.scala
│ │ ├── Compliance.scala
│ │ ├── Correlation.scala
│ │ ├── CountDistinct.scala
│ │ ├── CustomAggregator.scala
│ │ ├── CustomSql.scala
│ │ ├── DataType.scala
│ │ ├── DatasetMatchAnalyzer.scala
│ │ ├── DatasetMatchState.scala
│ │ ├── Distance.scala
│ │ ├── Distinctness.scala
│ │ ├── Entropy.scala
│ │ ├── ExactQuantile.scala
│ │ ├── FilterableAnalyzer.scala
│ │ ├── GroupingAnalyzers.scala
│ │ ├── Histogram.scala
│ │ ├── KLLSketch.scala
│ │ ├── MaxLength.scala
│ │ ├── Maximum.scala
│ │ ├── Mean.scala
│ │ ├── MinLength.scala
│ │ ├── Minimum.scala
│ │ ├── MutualInformation.scala
│ │ ├── NonSampleCompactor.scala
│ │ ├── PatternMatch.scala
│ │ ├── QuantileNonSample.scala
│ │ ├── RatioOfSums.scala
│ │ ├── Size.scala
│ │ ├── StandardDeviation.scala
│ │ ├── StateProvider.scala
│ │ ├── Sum.scala
│ │ ├── UniqueValueRatio.scala
│ │ ├── Uniqueness.scala
│ │ ├── applicability
│ │ │ └── Applicability.scala
│ │ ├── catalyst
│ │ │ ├── AttributeReferenceCreation.java
│ │ │ ├── DeequFunctions.scala
│ │ │ ├── HLLConstants.scala
│ │ │ ├── KLLSketchSerializer.scala
│ │ │ ├── StatefulApproxQuantile.scala
│ │ │ ├── StatefulCorrelation.scala
│ │ │ ├── StatefulDataType.scala
│ │ │ ├── StatefulHyperloglogPlus.scala
│ │ │ ├── StatefulKLLSketch.scala
│ │ │ └── StatefulStdDevPop.scala
│ │ └── runners
│ │ │ ├── AnalysisRunBuilder.scala
│ │ │ ├── AnalysisRunner.scala
│ │ │ ├── AnalyzerContext.scala
│ │ │ ├── KLLRunner.scala
│ │ │ └── MetricCalculationException.scala
│ │ ├── anomalydetection
│ │ ├── AbsoluteChangeStrategy.scala
│ │ ├── AnomalyDetectionStrategy.scala
│ │ ├── AnomalyDetector.scala
│ │ ├── BaseChangeStrategy.scala
│ │ ├── BatchNormalStrategy.scala
│ │ ├── DetectionResult.scala
│ │ ├── HistoryUtils.scala
│ │ ├── OnlineNormalStrategy.scala
│ │ ├── RateOfChangeStrategy.scala
│ │ ├── RelativeRateOfChangeStrategy.scala
│ │ ├── SimpleThresholdStrategy.scala
│ │ └── seasonal
│ │ │ └── HoltWinters.scala
│ │ ├── checks
│ │ ├── Check.scala
│ │ ├── CheckWithLastConstraintFilterable.scala
│ │ └── ColumnCondition.scala
│ │ ├── comparison
│ │ ├── ComparisonBase.scala
│ │ ├── ComparisonResult.scala
│ │ ├── DataSynchronization.scala
│ │ └── ReferentialIntegrity.scala
│ │ ├── constraints
│ │ ├── AnalysisBasedConstraint.scala
│ │ ├── ConstrainableDataTypes.scala
│ │ └── Constraint.scala
│ │ ├── dqdl
│ │ ├── EvaluateDataQuality.scala
│ │ ├── execution
│ │ │ ├── DQDLExecutor.scala
│ │ │ └── DefaultOperandEvaluator.scala
│ │ ├── model
│ │ │ └── ExecutableRule.scala
│ │ ├── translation
│ │ │ ├── DQDLRuleConverter.scala
│ │ │ ├── DQDLRuleTranslator.scala
│ │ │ └── DeequOutcomeTranslator.scala
│ │ └── util
│ │ │ ├── DQDLUtility.scala
│ │ │ └── DefaultDQDLParser.scala
│ │ ├── examples
│ │ ├── AnomalyDetectionExample.scala
│ │ ├── BasicExample.scala
│ │ ├── ConstraintSuggestionExample.scala
│ │ ├── DataProfilingExample.scala
│ │ ├── ExampleUtils.scala
│ │ ├── IncrementalMetricsExample.scala
│ │ ├── KLLCheckExample.scala
│ │ ├── KLLExample.scala
│ │ ├── MetricsRepositoryExample.scala
│ │ ├── UpdateMetricsOnPartitionedDataExample.scala
│ │ ├── algebraic_states_example.md
│ │ ├── anomaly_detection_example.md
│ │ ├── constraint_suggestion_example.md
│ │ ├── data_profiling_example.md
│ │ ├── entities.scala
│ │ └── metrics_repository_example.md
│ │ ├── io
│ │ └── DfsUtils.scala
│ │ ├── metrics
│ │ ├── HistogramMetric.scala
│ │ ├── KLLMetric.scala
│ │ └── Metric.scala
│ │ ├── profiles
│ │ ├── ColumnProfile.scala
│ │ ├── ColumnProfiler.scala
│ │ ├── ColumnProfilerRunBuilder.scala
│ │ └── ColumnProfilerRunner.scala
│ │ ├── repository
│ │ ├── AnalysisResult.scala
│ │ ├── AnalysisResultSerde.scala
│ │ ├── MetricsRepository.scala
│ │ ├── MetricsRepositoryMultipleResultsLoader.scala
│ │ ├── fs
│ │ │ └── FileSystemMetricsRepository.scala
│ │ ├── memory
│ │ │ └── InMemoryMetricsRepository.scala
│ │ └── sparktable
│ │ │ └── SparkMetricsRepository.scala
│ │ ├── schema
│ │ └── RowLevelSchemaValidator.scala
│ │ ├── suggestions
│ │ ├── ConstraintSuggestion.scala
│ │ ├── ConstraintSuggestionResult.scala
│ │ ├── ConstraintSuggestionRunBuilder.scala
│ │ ├── ConstraintSuggestionRunner.scala
│ │ └── rules
│ │ │ ├── CategoricalRangeRule.scala
│ │ │ ├── CompleteIfCompleteRule.scala
│ │ │ ├── ConstraintRule.scala
│ │ │ ├── FractionalCategoricalRangeRule.scala
│ │ │ ├── HasMax.scala
│ │ │ ├── HasMaxLength.scala
│ │ │ ├── HasMean.scala
│ │ │ ├── HasMin.scala
│ │ │ ├── HasMinLength.scala
│ │ │ ├── HasStandardDeviation.scala
│ │ │ ├── NonNegativeNumbersRule.scala
│ │ │ ├── RetainCompletenessRule.scala
│ │ │ ├── RetainTypeRule.scala
│ │ │ ├── UniqueIfApproximatelyUniqueRule.scala
│ │ │ └── interval
│ │ │ ├── ConfidenceIntervalStrategy.scala
│ │ │ ├── WaldIntervalStrategy.scala
│ │ │ └── WilsonScoreIntervalStrategy.scala
│ │ └── utilities
│ │ └── ColumnUtil.scala
└── test
│ ├── resources
│ ├── EMRSparkShellTest.scala
│ └── log4j.properties
│ └── scala
│ └── com
│ └── amazon
│ └── deequ
│ ├── DatatypeSuggestionTest.scala
│ ├── KLL
│ ├── KLLBenchmark.java
│ ├── KLLBenchmarkHelper.scala
│ ├── KLLDistanceTest.scala
│ ├── KLLProbTest.scala
│ └── KLLProfileTest.scala
│ ├── SparkBasicTest.scala
│ ├── SparkContextSpec.scala
│ ├── SparkMonitor.scala
│ ├── SuggestionAndVerificationIntegrationTest.scala
│ ├── VerificationResultTest.scala
│ ├── VerificationSuiteTest.scala
│ ├── analyzers
│ ├── AnalysisTest.scala
│ ├── AnalyzerTests.scala
│ ├── ColumnCountTest.scala
│ ├── CompletenessTest.scala
│ ├── ComplianceTest.scala
│ ├── CustomAggregatorTest.scala
│ ├── CustomSqlTest.scala
│ ├── DistinctnessTest.scala
│ ├── IncrementalAnalysisTest.scala
│ ├── IncrementalAnalyzerTest.scala
│ ├── MaxLengthTest.scala
│ ├── MaximumTest.scala
│ ├── MinLengthTest.scala
│ ├── MinimumTest.scala
│ ├── NullHandlingTests.scala
│ ├── PartitionedTableIntegrationTest.scala
│ ├── PatternMatchTest.scala
│ ├── StateAggregationIntegrationTest.scala
│ ├── StateAggregationTests.scala
│ ├── StateProviderTest.scala
│ ├── StatesTest.scala
│ ├── UniquenessTest.scala
│ └── runners
│ │ ├── AnalysisRunnerTests.scala
│ │ └── AnalyzerContextTest.scala
│ ├── anomalydetection
│ ├── AbsoluteChangeStrategyTest.scala
│ ├── AnomalyDetectionTestUtils.scala
│ ├── AnomalyDetectionTestUtilsTest.scala
│ ├── AnomalyDetectorTest.scala
│ ├── BatchNormalStrategyTest.scala
│ ├── HistoryUtilsTest.scala
│ ├── OnlineNormalStrategyTest.scala
│ ├── RateOfChangeStrategyTest.scala
│ ├── RelativeRateOfChangeStrategyTest.scala
│ ├── SimpleThresholdStrategyTest.scala
│ └── seasonal
│ │ └── HoltWintersTest.scala
│ ├── checks
│ ├── ApplicabilityTest.scala
│ ├── CheckTest.scala
│ ├── ColumnConditionTest.scala
│ └── FilterableCheckTest.scala
│ ├── comparison
│ ├── DataSynchronizationTest.scala
│ └── ReferentialIntegrityTest.scala
│ ├── constraints
│ ├── AnalysisBasedConstraintTest.scala
│ ├── ConstraintUtils.scala
│ └── ConstraintsTest.scala
│ ├── dqdl
│ ├── DefaultDQDLParserTest.scala
│ └── translation
│ │ └── DQDLRuleTranslatorSpec.scala
│ ├── examples
│ └── ExamplesTest.scala
│ ├── metrics
│ └── MetricsTests.scala
│ ├── package.scala
│ ├── profiles
│ ├── ColumnProfilerRunnerTest.scala
│ └── ColumnProfilerTest.scala
│ ├── repository
│ ├── AnalysisResultSerdeTest.scala
│ ├── AnalysisResultTest.scala
│ ├── MetricsRepositoryAnomalyDetectionIntegrationTest.scala
│ ├── MetricsRepositoryMultipleResultsLoaderTest.scala
│ ├── fs
│ │ └── FileSystemMetricsRepositoryTest.scala
│ ├── memory
│ │ └── InMemoryMetricsRepositoryTest.scala
│ └── sparktable
│ │ └── SparkTableMetricsRepositoryTest.scala
│ ├── schema
│ └── RowLevelSchemaValidatorTest.scala
│ ├── suggestions
│ ├── ConstraintSuggestionResultTest.scala
│ ├── ConstraintSuggestionRunnerTest.scala
│ ├── ConstraintSuggestionsIntegrationTest.scala
│ └── rules
│ │ ├── ConstraintRulesTest.scala
│ │ └── interval
│ │ └── IntervalStrategyTest.scala
│ └── utils
│ ├── AssertionUtils.scala
│ ├── CollectionUtils.scala
│ ├── ConditionUtils.scala
│ ├── FixtureSupport.scala
│ └── TempFileUtils.scala
└── test-data
├── README.md
└── titanic.csv
/.github/ISSUE_TEMPLATE/bug_report.md:
--------------------------------------------------------------------------------
1 | ---
2 | name: Bug report
3 | about: Create a report to help us improve
4 | title: "[BUG]"
5 | labels: bug
6 | assignees: ''
7 |
8 | ---
9 |
10 | **Describe the bug**
11 | A clear and concise description of what the bug is.
12 |
13 | **To Reproduce**
14 | Steps to reproduce the behavior:
15 | 1. Go to '...'
16 | 2. Click on '....'
17 | 3. Scroll down to '....'
18 | 4. See error
19 |
20 | **Expected behavior**
21 | A clear and concise description of what you expected to happen.
22 |
23 | **Screenshots**
24 | If applicable, add screenshots to help explain your problem.
25 |
26 | **Additional context**
27 | Add any other context about the problem here.
28 |
--------------------------------------------------------------------------------
/.github/ISSUE_TEMPLATE/feature_request.md:
--------------------------------------------------------------------------------
1 | ---
2 | name: Feature request
3 | about: Suggest an idea for this project
4 | title: "[FEATURE]"
5 | labels: enhancement
6 | assignees: ''
7 |
8 | ---
9 |
10 | **Is your feature request related to a problem? Please describe.**
11 | A clear and concise description of what the problem is. Ex. I'm always frustrated when [...]
12 |
13 | **Describe the solution you'd like**
14 | A clear and concise description of what you want to happen.
15 |
16 | **Describe alternatives you've considered**
17 | A clear and concise description of any alternative solutions or features you've considered.
18 |
19 | **Additional context**
20 | Add any other context or screenshots about the feature request here.
21 |
--------------------------------------------------------------------------------
/.github/ISSUE_TEMPLATE/general-inquiry.md:
--------------------------------------------------------------------------------
1 | ---
2 | name: General Inquiry
3 | about: Ask general questions about this project
4 | title: ''
5 | labels: question
6 | assignees: ''
7 |
8 | ---
9 |
10 | **Ask questions that don't apply to the other templates (Bug report, Feature request)**
11 |
--------------------------------------------------------------------------------
/.github/PULL_REQUEST_TEMPLATE.md:
--------------------------------------------------------------------------------
1 | *Issue #, if available:*
2 |
3 | *Description of changes:*
4 |
5 |
6 | By submitting this pull request, I confirm that my contribution is made under the terms of the Apache 2.0 license.
7 |
--------------------------------------------------------------------------------
/.github/workflows/maven.yml:
--------------------------------------------------------------------------------
1 | name: Java CI with Maven
2 |
3 | on:
4 | push:
5 | branches: [ "master" ]
6 | pull_request:
7 | branches: [ "master" ]
8 |
9 | jobs:
10 | build:
11 |
12 | runs-on: ubuntu-latest
13 |
14 | steps:
15 | - uses: actions/checkout@v3
16 | - name: Set up JDK 8
17 | uses: actions/setup-java@v3
18 | with:
19 | java-version: '8'
20 | distribution: 'corretto'
21 | cache: maven
22 | - name: Build with Maven
23 | run: mvn clean verify
24 |
25 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | .idea/
2 | *.iml
3 | **/*.iml
4 | target/
5 | .metals/
6 | .vscode/
7 | .bloop/
8 | .DS_Store
9 | .scalafmt.conf
10 | *.log
11 |
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | language: java
2 | jdk: oraclejdk8
3 | dist: trusty
4 |
5 | script: make build
6 |
--------------------------------------------------------------------------------
/CODE_OF_CONDUCT.md:
--------------------------------------------------------------------------------
1 | ## Code of Conduct
2 | This project has adopted the [Amazon Open Source Code of Conduct](https://aws.github.io/code-of-conduct).
3 | For more information see the [Code of Conduct FAQ](https://aws.github.io/code-of-conduct-faq) or contact
4 | opensource-codeofconduct@amazon.com with any additional questions or comments.
5 |
--------------------------------------------------------------------------------
/Makefile:
--------------------------------------------------------------------------------
1 | build:
2 | mvn clean install
3 | compile:
4 | mvn clean compile
--------------------------------------------------------------------------------
/NOTICE:
--------------------------------------------------------------------------------
1 | Deequ
2 | Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 |
--------------------------------------------------------------------------------
/docs/key-concepts.md:
--------------------------------------------------------------------------------
1 | # Key Concepts in the Codebase
2 | There are a few key concepts that will help you to understand the codebase.
3 |
4 | ## Metrics, Analyzers, and State
5 | Metrics represent some metric associated with the data that changes over time. For example counting the rows in a
6 | DataFrame.
7 |
8 | An Analyzer knows how to calculate a Metric based on some input DataFrame.
9 |
10 | State is an optimization - it represents the state of the data, from which a metric can be calculated. This intermediate
11 | state can then be used to calculate future metrics more quickly. Check out the examples for some further details.
12 |
13 | ## Overall flow of running deequ checks
14 | When running checks a user specifies a DataFrame and a number of checks to do on that DataFrame. Many checks in Deequ
15 | are based on metrics which describe the data. In order to perform the checks the user requests deequ follows the
16 | following process:
17 | * First deequ figures out which Analyzers are required
18 | * Metrics are calculated using those Analyzers:
19 | * Metrics are also stored if a MetricsRepository is provided
20 | * Intermediate state is stored if a StatePersister is provided
21 | * Intermediate state is used for metric calculations if a StateLoader is provided
22 | * Checks are evaluated using the calculated Metrics
23 |
24 | The reason it works this way is for performance, primarily because calculating metrics at the same time gives the
25 | opportunity to calculate them in fewer passes over the data.
26 |
27 | ### Analyzers
28 | Types of analyzers:
29 | * ScanShareableAnalyzer - an analyzer which computes a metric based on a straight scan over the data, without any
30 | grouping being required
31 | * GroupingAnalyzer - an analyzer that requires the data to be grouped by a set of columns before the metric can be
32 | calculated
33 |
34 | ### Metrics
35 | A metric includes the following key details:
36 | * name - the name for the type of metric
37 | * entity - the type of entity the metric is recorded against. e.g. A column, dataset, or multicolumn
38 | * instance - information about this instance of the metric. For example this could be the column name the metric is
39 | operating on
40 | * value - the value of the metric at a point in time. The type of this value varies between metrics.
41 |
42 | #### Metrics storage
43 | Metrics can be stored in a metrics repository. An entry in the repository consists of:
44 | * A resultKey, which is a combination of a timestamp and a map of tags. Typically a user may want to record things
45 | like the data source (e.g. table name) with the tags. The resultKey can be used to lookup stored metrics
46 | * An analyzerContext, which consists of a map of Analyzers to Metrics
47 |
48 | ### State
49 | Please consult the examples or the codebase for more details on State.
50 |
--------------------------------------------------------------------------------
/settings.xml:
--------------------------------------------------------------------------------
1 |
2 |
3 |
4 | ossrh
5 | ${env.MAVEN_REPO_USERNAME}
6 | ${env.MAVEN_REPO_PASSWORD}
7 |
8 |
9 |
10 |
11 | release
12 |
13 | true
14 |
15 |
16 | gpg
17 | DBF0F18920DAB3377B638B5579288555449C57B1
18 |
19 |
20 |
21 |
22 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Analysis.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.runners.{AnalysisRunner, AnalyzerContext}
20 | import com.amazon.deequ.metrics.Metric
21 | import org.apache.spark.sql.DataFrame
22 | import org.apache.spark.storage.StorageLevel
23 |
24 | /**
25 | * Defines a set of analyzers to run on data.
26 | *
27 | * @param analyzers
28 | */
29 | case class Analysis(analyzers: Seq[Analyzer[_, Metric[_]]] = Seq.empty) {
30 |
31 | def addAnalyzer(analyzer: Analyzer[_, Metric[_]]): Analysis = {
32 | Analysis(analyzers :+ analyzer)
33 | }
34 |
35 | def addAnalyzers(otherAnalyzers: Seq[Analyzer[_, Metric[_]]]): Analysis = {
36 | Analysis(analyzers ++ otherAnalyzers)
37 | }
38 |
39 | /**
40 | * Compute the metrics from the analyzers configured in the analyis
41 | *
42 | * @param data data on which to operate
43 | * @param aggregateWith load existing states for the configured analyzers and aggregate them
44 | * (optional)
45 | * @param saveStatesWith persist resulting states for the configured analyzers (optional)
46 | * @param storageLevelOfGroupedDataForMultiplePasses caching level for grouped data that must
47 | * be accessed multiple times (use
48 | * StorageLevel.NONE to completely disable
49 | * caching)
50 | * @return
51 | */
52 | @deprecated("Use the AnalysisRunner instead (the onData method there)", "24-09-2019")
53 | def run(
54 | data: DataFrame,
55 | aggregateWith: Option[StateLoader] = None,
56 | saveStatesWith: Option[StatePersister] = None,
57 | storageLevelOfGroupedDataForMultiplePasses: StorageLevel = StorageLevel.MEMORY_AND_DISK)
58 | : AnalyzerContext = {
59 |
60 | AnalysisRunner.doAnalysisRun(data, analyzers, aggregateWith = aggregateWith,
61 | saveStatesWith = saveStatesWith)
62 | }
63 | }
64 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/ApproxCountDistinct.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Preconditions.hasColumn
20 | import org.apache.spark.sql.DeequFunctions.stateful_approx_count_distinct
21 | import org.apache.spark.sql.catalyst.expressions.aggregate.DeequHyperLogLogPlusPlusUtils
22 | import org.apache.spark.sql.types.StructType
23 | import org.apache.spark.sql.{Column, Row}
24 | import Analyzers._
25 |
26 | case class ApproxCountDistinctState(words: Array[Long])
27 | extends DoubleValuedState[ApproxCountDistinctState] {
28 |
29 | override def sum(other: ApproxCountDistinctState): ApproxCountDistinctState = {
30 | ApproxCountDistinctState(DeequHyperLogLogPlusPlusUtils.merge(words, other.words))
31 | }
32 |
33 | override def metricValue(): Double = {
34 | DeequHyperLogLogPlusPlusUtils.count(words)
35 | }
36 |
37 | override def toString: String = {
38 | s"ApproxCountDistinctState(${words.mkString(",")})"
39 | }
40 | }
41 |
42 | /**
43 | * Compute approximated count distinct with HyperLogLogPlusPlus.
44 | *
45 | * @param column Which column to compute this aggregation on.
46 | */
47 | case class ApproxCountDistinct(column: String, where: Option[String] = None)
48 | extends StandardScanShareableAnalyzer[ApproxCountDistinctState]("ApproxCountDistinct", column)
49 | with FilterableAnalyzer {
50 |
51 | override def aggregationFunctions(): Seq[Column] = {
52 | stateful_approx_count_distinct(conditionalSelection(column, where)) :: Nil
53 | }
54 |
55 | override def fromAggregationResult(result: Row, offset: Int): Option[ApproxCountDistinctState] = {
56 |
57 | ifNoNullsIn(result, offset) { _ =>
58 | DeequHyperLogLogPlusPlusUtils.wordsFromBytes(result.getAs[Array[Byte]](offset))
59 | }
60 | }
61 |
62 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
63 | hasColumn(column) :: Nil
64 | }
65 |
66 | override def filterCondition: Option[String] = where
67 | }
68 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/ColumnCount.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
12 | * either express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | *
16 | */
17 |
18 | package com.amazon.deequ.analyzers
19 |
20 | import com.amazon.deequ.metrics.DoubleMetric
21 | import com.amazon.deequ.metrics.Entity
22 | import org.apache.spark.sql.DataFrame
23 |
24 | case class ColumnCount() extends Analyzer[NumMatches, DoubleMetric] {
25 |
26 | val name = "ColumnCount"
27 | val instance = "*"
28 | val entity = Entity.Dataset
29 |
30 | /**
31 | * Compute the state (sufficient statistics) from the data
32 | *
33 | * @param data the input dataframe
34 | * @return the number of columns in the input
35 | */
36 | override def computeStateFrom(data: DataFrame, filterCondition: Option[String]): Option[NumMatches] = {
37 | if (filterCondition.isDefined) {
38 | throw new IllegalArgumentException("ColumnCount does not accept a filter condition")
39 | } else {
40 | val numColumns = data.columns.size
41 | Some(NumMatches(numColumns))
42 | }
43 | }
44 |
45 | /**
46 | * Compute the metric from the state (sufficient statistics)
47 | *
48 | * @param state the computed state from [[computeStateFrom]]
49 | * @return a double metric indicating the number of columns for this analyzer
50 | */
51 | override def computeMetricFrom(state: Option[NumMatches]): DoubleMetric = {
52 | state
53 | .map(v => Analyzers.metricFromValue(v.metricValue(), name, instance, entity))
54 | .getOrElse(Analyzers.metricFromEmpty(this, name, instance, entity))
55 | }
56 |
57 | /**
58 | * Compute the metric from a failure - reports the exception thrown while trying to count columns
59 | */
60 | override private[deequ] def toFailureMetric(failure: Exception): DoubleMetric = {
61 | Analyzers.metricFromFailure(failure, name, instance, entity)
62 | }
63 | }
64 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Completeness.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isNotNested}
20 | import org.apache.spark.sql.functions.sum
21 | import org.apache.spark.sql.types.{IntegerType, StructType}
22 | import Analyzers._
23 | import com.google.common.annotations.VisibleForTesting
24 | import org.apache.spark.sql.functions.col
25 | import org.apache.spark.sql.functions.expr
26 | import org.apache.spark.sql.{Column, Row}
27 |
28 | /** Completeness is the fraction of non-null values in a column of a DataFrame. */
29 | case class Completeness(column: String, where: Option[String] = None,
30 | analyzerOptions: Option[AnalyzerOptions] = None) extends
31 | StandardScanShareableAnalyzer[NumMatchesAndCount]("Completeness", column) with
32 | FilterableAnalyzer {
33 |
34 | override def fromAggregationResult(result: Row, offset: Int): Option[NumMatchesAndCount] = {
35 | ifNoNullsIn(result, offset, howMany = 2) { _ =>
36 | NumMatchesAndCount(result.getLong(offset), result.getLong(offset + 1), Some(rowLevelResults))
37 | }
38 | }
39 |
40 | override def aggregationFunctions(): Seq[Column] = {
41 |
42 | val summation = sum(criterion.cast(IntegerType))
43 |
44 | summation :: conditionalCount(where) :: Nil
45 | }
46 |
47 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
48 | hasColumn(column) :: isNotNested(column) :: Nil
49 | }
50 |
51 | override def filterCondition: Option[String] = where
52 |
53 | @VisibleForTesting // required by some tests that compare analyzer results to an expected state
54 | private[deequ] def criterion: Column = conditionalSelection(column, where).isNotNull
55 |
56 | private[deequ] def rowLevelResults: Column = {
57 | val whereCondition = where.map { expression => expr(expression)}
58 | conditionalSelectionFilteredFromColumns(
59 | col(column).isNotNull, whereCondition, getRowLevelFilterTreatment(analyzerOptions))
60 | }
61 | }
62 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/CountDistinct.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers._
20 | import com.amazon.deequ.metrics.DoubleMetric
21 | import org.apache.spark.sql.Column
22 | import org.apache.spark.sql.Row
23 | import org.apache.spark.sql.functions.count
24 |
25 | case class CountDistinct(columns: Seq[String])
26 | extends ScanShareableFrequencyBasedAnalyzer("CountDistinct", columns) {
27 |
28 | override def aggregationFunctions(numRows: Long): Seq[Column] = {
29 | count("*") :: Nil
30 | }
31 |
32 | override def fromAggregationResult(result: Row, offset: Int, fullColumn: Option[Column] = None): DoubleMetric = {
33 | toSuccessMetric(result.getLong(offset).toDouble, None)
34 | }
35 | }
36 |
37 | object CountDistinct {
38 | def apply(column: String): CountDistinct = {
39 | new CountDistinct(column :: Nil)
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/CustomAggregator.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2024 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
4 | * use this file except in compliance with the License. A copy of the License
5 | * is located at
6 | *
7 | * http://aws.amazon.com/apache2.0/
8 | *
9 | * or in the "license" file accompanying this file. This file is distributed on
10 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
11 | * express or implied. See the License for the specific language governing
12 | * permissions and limitations under the License.
13 | *
14 | */
15 | package com.amazon.deequ.analyzers
16 |
17 | import com.amazon.deequ.metrics.AttributeDoubleMetric
18 | import com.amazon.deequ.metrics.Entity
19 | import org.apache.spark.sql.DataFrame
20 |
21 | import scala.util.Failure
22 | import scala.util.Success
23 | import scala.util.Try
24 |
25 | // Define a custom state to hold aggregation results
26 | case class AggregatedMetricState(counts: Map[String, Int], total: Int)
27 | extends DoubleValuedState[AggregatedMetricState] {
28 |
29 | override def sum(other: AggregatedMetricState): AggregatedMetricState = {
30 | val combinedCounts = counts ++ other
31 | .counts
32 | .map { case (k, v) => k -> (v + counts.getOrElse(k, 0)) }
33 | AggregatedMetricState(combinedCounts, total + other.total)
34 | }
35 |
36 | override def metricValue(): Double = counts.values.sum.toDouble / total
37 | }
38 |
39 | // Define the analyzer
40 | case class CustomAggregator(aggregatorFunc: DataFrame => AggregatedMetricState,
41 | metricName: String,
42 | instance: String = "Dataset")
43 | extends Analyzer[AggregatedMetricState, AttributeDoubleMetric] {
44 |
45 | override def computeStateFrom(data: DataFrame, filterCondition: Option[String] = None)
46 | : Option[AggregatedMetricState] = {
47 | Try(aggregatorFunc(data)) match {
48 | case Success(state) => Some(state)
49 | case Failure(_) => None
50 | }
51 | }
52 |
53 | override def computeMetricFrom(state: Option[AggregatedMetricState]): AttributeDoubleMetric = {
54 | state match {
55 | case Some(detState) =>
56 | val metrics = detState.counts.map { case (key, count) =>
57 | key -> (count.toDouble / detState.total)
58 | }
59 | AttributeDoubleMetric(Entity.Column, metricName, instance, Success(metrics))
60 | case None =>
61 | AttributeDoubleMetric(Entity.Column, metricName, instance,
62 | Failure(new RuntimeException("Metric computation failed")))
63 | }
64 | }
65 |
66 | override private[deequ] def toFailureMetric(failure: Exception): AttributeDoubleMetric = {
67 | AttributeDoubleMetric(Entity.Column, metricName, instance, Failure(failure))
68 | }
69 | }
70 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/DatasetMatchState.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2024 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | /**
20 | * Represents the state of datasetMatch between two DataFrames in Deequ.
21 | * This state keeps track of the count of matched record count and the total record count.
22 | * It measures how well the data in the two DataFrames matches.
23 | *
24 | * @param matchedDataCount The count of records that are considered match between the two DataFrames.
25 | * @param totalDataCount The total count of records for check.
26 | *
27 | * The `sum` method allows for aggregation of this state with another, combining the counts from both states.
28 | * This is useful in distributed computations where states from different partitions need to be aggregated.
29 | *
30 | * The `metricValue` method computes the synchronization ratio. It is the ratio of `matchedDataCount` to `dataCount`.
31 | * If `dataCount` is zero, which means no data points were examined, the method returns `Double.NaN` to indicate
32 | * the undefined state.
33 | *
34 | */
35 | case class DatasetMatchState(matchedDataCount: Long, totalDataCount: Long)
36 | extends DoubleValuedState[DatasetMatchState] {
37 | override def sum(other: DatasetMatchState): DatasetMatchState = {
38 | DatasetMatchState(matchedDataCount + other.matchedDataCount, totalDataCount + other.totalDataCount)
39 | }
40 |
41 | override def metricValue(): Double = {
42 | if (totalDataCount == 0L) Double.NaN else matchedDataCount.toDouble / totalDataCount.toDouble
43 | }
44 | }
45 |
46 | object DatasetMatchState
47 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Distinctness.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers.COUNT_COL
20 | import com.amazon.deequ.metrics.DoubleMetric
21 | import org.apache.spark.sql.functions.{col, sum}
22 | import org.apache.spark.sql.types.DoubleType
23 | import org.apache.spark.sql.Column
24 | import org.apache.spark.sql.Row
25 |
26 | /**
27 | * Distinctness is the fraction of distinct values of a column(s).
28 | *
29 | * @param columns the column(s) for which to compute distinctness
30 | */
31 | case class Distinctness(columns: Seq[String], where: Option[String] = None)
32 | extends ScanShareableFrequencyBasedAnalyzer("Distinctness", columns)
33 | with FilterableAnalyzer {
34 |
35 | override def aggregationFunctions(numRows: Long): Seq[Column] = {
36 | (sum(col(COUNT_COL).geq(1).cast(DoubleType)) / numRows) :: Nil
37 | }
38 |
39 | override def fromAggregationResult(result: Row, offset: Int, fullColumn: Option[Column]): DoubleMetric = {
40 | super.fromAggregationResult(result, offset, None)
41 | }
42 |
43 | override def filterCondition: Option[String] = where
44 | }
45 |
46 | object Distinctness {
47 | def apply(column: String): Distinctness = {
48 | new Distinctness(column :: Nil)
49 | }
50 | }
51 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Entropy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers.COUNT_COL
20 | import com.amazon.deequ.metrics.DoubleMetric
21 | import org.apache.spark.sql.Column
22 | import org.apache.spark.sql.Row
23 | import org.apache.spark.sql.functions.{col, sum, udf}
24 |
25 | /**
26 | * Entropy is a measure of the level of information contained in a message. Given the probability
27 | * distribution over values in a column, it describes how many bits are required to identify a
28 | * value.
29 | */
30 | case class Entropy(column: String, where: Option[String] = None)
31 | extends ScanShareableFrequencyBasedAnalyzer("Entropy", column :: Nil)
32 | with FilterableAnalyzer {
33 |
34 | override def aggregationFunctions(numRows: Long): Seq[Column] = {
35 | val summands = udf { (count: Double) =>
36 | if (count == 0.0) {
37 | 0.0
38 | } else {
39 | -(count / numRows) * math.log(count / numRows)
40 | }
41 | }
42 |
43 | sum(summands(col(COUNT_COL))) :: Nil
44 | }
45 |
46 | override def fromAggregationResult(result: Row, offset: Int, fullColumn: Option[Column] = None): DoubleMetric = {
47 | super.fromAggregationResult(result, offset, None)
48 | }
49 |
50 | override def filterCondition: Option[String] = where
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/ExactQuantile.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isNumeric}
20 | import com.amazon.deequ.analyzers.Analyzers.{conditionalSelection, ifNoNullsIn}
21 | import com.amazon.deequ.metrics.FullColumn
22 | import org.apache.curator.shaded.com.google.common.annotations.VisibleForTesting
23 | import org.apache.spark.sql.{Column, Row}
24 | import org.apache.spark.sql.functions.expr
25 | import org.apache.spark.sql.types.{DoubleType, StructType}
26 |
27 | case class ExactQuantileState(exactQuantile: Double, quantile: Double, override val fullColumn: Option[Column] = None)
28 | extends DoubleValuedState[ExactQuantileState] with FullColumn {
29 | override def sum(other: ExactQuantileState): ExactQuantileState = {
30 |
31 | ExactQuantileState(
32 | expr(s"percentile($fullColumn, $quantile)").toString().toDouble,
33 | quantile,
34 | sum(fullColumn, other.fullColumn))
35 | }
36 |
37 | override def metricValue(): Double = {
38 | exactQuantile
39 | }
40 | }
41 |
42 | case class ExactQuantile(column: String,
43 | quantile: Double,
44 | where: Option[String] = None)
45 | extends StandardScanShareableAnalyzer[ExactQuantileState]("ExactQuantile", column)
46 | with FilterableAnalyzer {
47 | override def aggregationFunctions(): Seq[Column] = {
48 | expr(s"percentile(${conditionalSelection(column, where).cast(DoubleType)}, $quantile)") :: Nil
49 | }
50 |
51 | override def fromAggregationResult(result: Row, offset: Int): Option[ExactQuantileState] = {
52 | ifNoNullsIn(result, offset) { _ =>
53 | ExactQuantileState(result.getDouble(offset), quantile, Some(criterion))
54 | }
55 | }
56 |
57 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
58 | hasColumn(column) :: isNumeric(column) :: Nil
59 | }
60 |
61 | override def filterCondition: Option[String] = where
62 |
63 | @VisibleForTesting
64 | private def criterion: Column = conditionalSelection(column, where).cast(DoubleType)
65 | }
66 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/FilterableAnalyzer.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | /**
20 | * Common trait for Analyzers that support dataset filtering
21 | */
22 | trait FilterableAnalyzer {
23 | def filterCondition: Option[String]
24 | }
25 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/MaxLength.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers._
20 | import com.amazon.deequ.analyzers.NullBehavior.NullBehavior
21 | import com.amazon.deequ.analyzers.Preconditions.hasColumn
22 | import com.amazon.deequ.analyzers.Preconditions.isString
23 | import org.apache.spark.sql.Column
24 | import org.apache.spark.sql.Row
25 | import org.apache.spark.sql.functions.col
26 | import org.apache.spark.sql.functions.element_at
27 | import org.apache.spark.sql.functions.length
28 | import org.apache.spark.sql.functions.lit
29 | import org.apache.spark.sql.functions.max
30 | import org.apache.spark.sql.functions.when
31 | import org.apache.spark.sql.types.DoubleType
32 | import org.apache.spark.sql.types.StructType
33 |
34 | case class MaxLength(column: String, where: Option[String] = None, analyzerOptions: Option[AnalyzerOptions] = None)
35 | extends StandardScanShareableAnalyzer[MaxState]("MaxLength", column)
36 | with FilterableAnalyzer {
37 |
38 | override def aggregationFunctions(): Seq[Column] = {
39 | // The criterion returns a column where each row contains an array of 2 elements.
40 | // The first element of the array is a string that indicates if the row is "in scope" or "filtered" out.
41 | // The second element is the value used for calculating the metric. We use "element_at" to extract it.
42 | max(element_at(criterion, 2).cast(DoubleType)) :: Nil
43 | }
44 |
45 | override def fromAggregationResult(result: Row, offset: Int): Option[MaxState] = {
46 | ifNoNullsIn(result, offset) { _ =>
47 | MaxState(result.getDouble(offset), Some(criterion))
48 | }
49 | }
50 |
51 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
52 | hasColumn(column):: isString(column) :: Nil
53 | }
54 |
55 | override def filterCondition: Option[String] = where
56 |
57 | private[deequ] def criterion: Column = {
58 | val isNullCheck = col(column).isNull
59 | val colLength = length(col(column)).cast(DoubleType)
60 | val updatedColumn = getNullBehavior match {
61 | case NullBehavior.Fail => when(isNullCheck, Double.MaxValue).otherwise(colLength)
62 | // Empty String is 0 length string
63 | case NullBehavior.EmptyString => when(isNullCheck, lit(0.0)).otherwise(colLength)
64 | case NullBehavior.Ignore => colLength
65 | }
66 |
67 | conditionalSelectionWithAugmentedOutcome(updatedColumn, where)
68 | }
69 |
70 | private def getNullBehavior: NullBehavior = {
71 | analyzerOptions
72 | .map { options => options.nullBehavior }
73 | .getOrElse(NullBehavior.Ignore)
74 | }
75 | }
76 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Maximum.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers._
20 | import com.amazon.deequ.analyzers.Preconditions.hasColumn
21 | import com.amazon.deequ.analyzers.Preconditions.isNumeric
22 | import com.amazon.deequ.metrics.FullColumn
23 | import com.google.common.annotations.VisibleForTesting
24 | import org.apache.spark.sql.functions.col
25 | import org.apache.spark.sql.functions.element_at
26 | import org.apache.spark.sql.functions.max
27 | import org.apache.spark.sql.types.DoubleType
28 | import org.apache.spark.sql.types.StructType
29 | import org.apache.spark.sql.Column
30 | import org.apache.spark.sql.Row
31 |
32 | case class MaxState(maxValue: Double, override val fullColumn: Option[Column] = None)
33 | extends DoubleValuedState[MaxState] with FullColumn {
34 |
35 | override def sum(other: MaxState): MaxState = {
36 | MaxState(math.max(maxValue, other.maxValue), sum(fullColumn, other.fullColumn))
37 | }
38 |
39 | override def metricValue(): Double = {
40 | maxValue
41 | }
42 | }
43 |
44 | case class Maximum(column: String, where: Option[String] = None, analyzerOptions: Option[AnalyzerOptions] = None)
45 | extends StandardScanShareableAnalyzer[MaxState]("Maximum", column)
46 | with FilterableAnalyzer {
47 |
48 | override def aggregationFunctions(): Seq[Column] = {
49 | // The criterion returns a column where each row contains an array of 2 elements.
50 | // The first element of the array is a string that indicates if the row is "in scope" or "filtered" out.
51 | // The second element is the value used for calculating the metric. We use "element_at" to extract it.
52 | max(element_at(criterion, 2).cast(DoubleType)) :: Nil
53 | }
54 |
55 | override def fromAggregationResult(result: Row, offset: Int): Option[MaxState] = {
56 | ifNoNullsIn(result, offset) { _ =>
57 | MaxState(result.getDouble(offset), Some(criterion))
58 | }
59 | }
60 |
61 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
62 | hasColumn(column) :: isNumeric(column) :: Nil
63 | }
64 |
65 | override def filterCondition: Option[String] = where
66 |
67 | @VisibleForTesting
68 | private def criterion: Column = conditionalSelectionWithAugmentedOutcome(col(column), where)
69 | }
70 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Mean.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isNumeric}
20 | import org.apache.spark.sql.{Column, Row}
21 | import org.apache.spark.sql.functions.{count, sum}
22 | import org.apache.spark.sql.types.{DoubleType, StructType, LongType}
23 | import Analyzers._
24 |
25 | case class MeanState(sum: Double, count: Long) extends DoubleValuedState[MeanState] {
26 |
27 | override def sum(other: MeanState): MeanState = {
28 | MeanState(sum + other.sum, count + other.count)
29 | }
30 |
31 | override def metricValue(): Double = {
32 | if (count == 0L) Double.NaN else sum / count
33 | }
34 | }
35 |
36 | case class Mean(column: String, where: Option[String] = None)
37 | extends StandardScanShareableAnalyzer[MeanState]("Mean", column)
38 | with FilterableAnalyzer {
39 |
40 | override def aggregationFunctions(): Seq[Column] = {
41 | sum(conditionalSelection(column, where)).cast(DoubleType) ::
42 | count(conditionalSelection(column, where)).cast(LongType) :: Nil
43 | }
44 |
45 | override def fromAggregationResult(result: Row, offset: Int): Option[MeanState] = {
46 |
47 | ifNoNullsIn(result, offset, howMany = 2) { _ =>
48 | MeanState(result.getDouble(offset), result.getLong(offset + 1))
49 | }
50 | }
51 |
52 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
53 | hasColumn(column) :: isNumeric(column) :: Nil
54 | }
55 |
56 | override def filterCondition: Option[String] = where
57 | }
58 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/MinLength.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers._
20 | import com.amazon.deequ.analyzers.NullBehavior.NullBehavior
21 | import com.amazon.deequ.analyzers.Preconditions.hasColumn
22 | import com.amazon.deequ.analyzers.Preconditions.isString
23 | import org.apache.spark.sql.Column
24 | import org.apache.spark.sql.Row
25 | import org.apache.spark.sql.functions.col
26 | import org.apache.spark.sql.functions.element_at
27 | import org.apache.spark.sql.functions.length
28 | import org.apache.spark.sql.functions.lit
29 | import org.apache.spark.sql.functions.min
30 | import org.apache.spark.sql.functions.when
31 | import org.apache.spark.sql.types.DoubleType
32 | import org.apache.spark.sql.types.StructType
33 |
34 | case class MinLength(column: String, where: Option[String] = None, analyzerOptions: Option[AnalyzerOptions] = None)
35 | extends StandardScanShareableAnalyzer[MinState]("MinLength", column)
36 | with FilterableAnalyzer {
37 |
38 | override def aggregationFunctions(): Seq[Column] = {
39 | // The criterion returns a column where each row contains an array of 2 elements.
40 | // The first element of the array is a string that indicates if the row is "in scope" or "filtered" out.
41 | // The second element is the value used for calculating the metric. We use "element_at" to extract it.
42 | min(element_at(criterion, 2).cast(DoubleType)) :: Nil
43 | }
44 |
45 | override def fromAggregationResult(result: Row, offset: Int): Option[MinState] = {
46 | ifNoNullsIn(result, offset) { _ =>
47 | MinState(result.getDouble(offset), Some(criterion))
48 | }
49 | }
50 |
51 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
52 | hasColumn(column) :: isString(column) :: Nil
53 | }
54 |
55 | override def filterCondition: Option[String] = where
56 |
57 | private[deequ] def criterion: Column = {
58 | val isNullCheck = col(column).isNull
59 | val colLength = length(col(column)).cast(DoubleType)
60 | val updatedColumn = getNullBehavior match {
61 | case NullBehavior.Fail => when(isNullCheck, Double.MinValue).otherwise(colLength)
62 | // Empty String is 0 length string
63 | case NullBehavior.EmptyString => when(isNullCheck, lit(0.0)).otherwise(colLength)
64 | case NullBehavior.Ignore => colLength
65 | }
66 |
67 | conditionalSelectionWithAugmentedOutcome(updatedColumn, where)
68 | }
69 |
70 | private def getNullBehavior: NullBehavior = {
71 | analyzerOptions
72 | .map { options => options.nullBehavior }
73 | .getOrElse(NullBehavior.Ignore)
74 | }
75 | }
76 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Minimum.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers._
20 | import com.amazon.deequ.analyzers.Preconditions.hasColumn
21 | import com.amazon.deequ.analyzers.Preconditions.isNumeric
22 | import com.amazon.deequ.metrics.FullColumn
23 | import com.google.common.annotations.VisibleForTesting
24 | import org.apache.spark.sql.functions.col
25 | import org.apache.spark.sql.functions.element_at
26 | import org.apache.spark.sql.functions.min
27 | import org.apache.spark.sql.types.DoubleType
28 | import org.apache.spark.sql.types.StructType
29 | import org.apache.spark.sql.Column
30 | import org.apache.spark.sql.Row
31 |
32 | case class MinState(minValue: Double, override val fullColumn: Option[Column] = None)
33 | extends DoubleValuedState[MinState] with FullColumn {
34 |
35 | override def sum(other: MinState): MinState = {
36 | MinState(math.min(minValue, other.minValue), sum(fullColumn, other.fullColumn))
37 | }
38 |
39 | override def metricValue(): Double = {
40 | minValue
41 | }
42 | }
43 |
44 | case class Minimum(column: String, where: Option[String] = None, analyzerOptions: Option[AnalyzerOptions] = None)
45 | extends StandardScanShareableAnalyzer[MinState]("Minimum", column)
46 | with FilterableAnalyzer {
47 |
48 | override def aggregationFunctions(): Seq[Column] = {
49 | // The criterion returns a column where each row contains an array of 2 elements.
50 | // The first element of the array is a string that indicates if the row is "in scope" or "filtered" out.
51 | // The second element is the value used for calculating the metric. We use "element_at" to extract it.
52 | min(element_at(criterion, 2).cast(DoubleType)) :: Nil
53 | }
54 |
55 | override def fromAggregationResult(result: Row, offset: Int): Option[MinState] = {
56 | ifNoNullsIn(result, offset) { _ =>
57 | MinState(result.getDouble(offset), Some(criterion))
58 | }
59 | }
60 |
61 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
62 | hasColumn(column) :: isNumeric(column) :: Nil
63 | }
64 |
65 | override def filterCondition: Option[String] = where
66 |
67 | @VisibleForTesting
68 | private def criterion: Column = conditionalSelectionWithAugmentedOutcome(col(column), where)
69 | }
70 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/NonSampleCompactor.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import scala.collection.mutable.ArrayBuffer
20 | import scala.reflect.ClassTag
21 | import scala.util.Random
22 |
23 | /**
24 | * A quantile sketcher whose output is half the size of its input.
25 | *
26 | * @tparam T type of the items being sketched. There should an ordering
27 | * over this item type
28 | */
29 | class NonSampleCompactor[T]()
30 | (implicit ordering: Ordering[T],
31 | ct: ClassTag[T])
32 | extends Serializable {
33 |
34 | var numOfCompress = 0
35 | var offset = 0
36 | var buffer: ArrayBuffer[T] = ArrayBuffer[T]()
37 |
38 | private def findOdd(items: Int): Option[T] = items % 2 match {
39 | case 1 => Some(buffer(math.max(items - 1, 0)))
40 | case _ => None
41 | }
42 |
43 | def compact : Array[T] = {
44 | var items = buffer.length
45 | val len = items - (items % 2)
46 | if (numOfCompress % 2 == 1) {
47 | offset = 1 - offset
48 | }
49 | // else {
50 | // offset = if (Random.nextBoolean()) 1 else 0
51 | // }
52 | val sortedBuffer = buffer.toArray.slice(0, len).sorted
53 |
54 | /** Selects half of the items from this level compactor to the next level compactor.
55 | * e.g. if sortedBuffer is Array(1,2,3,4), if offset is 1, output = Array(2,4),
56 | * and if offset is 0, output = Array(1,3), this will be the input to the next level compactor.
57 | */
58 | val output = (offset until len by 2).map(sortedBuffer(_)).toArray
59 | val tail = findOdd(items)
60 | items = items % 2
61 | var newBuffer = ArrayBuffer[T]()
62 | if (tail.isDefined) {
63 | newBuffer = newBuffer :+ tail.get
64 | }
65 | buffer = newBuffer
66 | numOfCompress = numOfCompress + 1
67 | output
68 | }
69 | }
70 |
71 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Size.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.metrics.Entity
20 | import org.apache.spark.sql.{Column, Row}
21 | import Analyzers._
22 |
23 | case class NumMatches(numMatches: Long) extends DoubleValuedState[NumMatches] {
24 |
25 | override def sum(other: NumMatches): NumMatches = {
26 | NumMatches(numMatches + other.numMatches)
27 | }
28 |
29 | override def metricValue(): Double = {
30 | numMatches.toDouble
31 | }
32 |
33 | }
34 |
35 | /** Size is the number of rows in a DataFrame. */
36 | case class Size(where: Option[String] = None)
37 | extends StandardScanShareableAnalyzer[NumMatches]("Size", "*", Entity.Dataset)
38 | with FilterableAnalyzer {
39 |
40 | override def aggregationFunctions(): Seq[Column] = {
41 | conditionalCount(where) :: Nil
42 | }
43 |
44 | override def fromAggregationResult(result: Row, offset: Int): Option[NumMatches] = {
45 | ifNoNullsIn(result, offset) { _ =>
46 | NumMatches(result.getLong(offset))
47 | }
48 | }
49 |
50 | override def filterCondition: Option[String] = where
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/StandardDeviation.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isNumeric}
20 | import org.apache.spark.sql.DeequFunctions.stateful_stddev_pop
21 | import org.apache.spark.sql.{Column, Row}
22 | import org.apache.spark.sql.types.StructType
23 | import Analyzers._
24 |
25 | case class StandardDeviationState(
26 | n: Double,
27 | avg: Double,
28 | m2: Double)
29 | extends DoubleValuedState[StandardDeviationState] {
30 |
31 | require(n > 0.0, "Standard deviation is undefined for n = 0.")
32 |
33 | override def metricValue(): Double = {
34 | math.sqrt(m2 / n)
35 | }
36 |
37 | override def sum(other: StandardDeviationState): StandardDeviationState = {
38 | val newN = n + other.n
39 | val delta = other.avg - avg
40 | val deltaN = if (newN == 0.0) 0.0 else delta / newN
41 |
42 | StandardDeviationState(newN, avg + deltaN * other.n,
43 | m2 + other.m2 + delta * deltaN * n * other.n)
44 | }
45 | }
46 |
47 | case class StandardDeviation(column: String, where: Option[String] = None)
48 | extends StandardScanShareableAnalyzer[StandardDeviationState]("StandardDeviation", column)
49 | with FilterableAnalyzer {
50 |
51 | override def aggregationFunctions(): Seq[Column] = {
52 | stateful_stddev_pop(conditionalSelection(column, where)) :: Nil
53 | }
54 |
55 | override def fromAggregationResult(result: Row, offset: Int): Option[StandardDeviationState] = {
56 |
57 | if (result.isNullAt(offset)) {
58 | None
59 | } else {
60 | val row = result.getAs[Row](offset)
61 | val n = row.getDouble(0)
62 |
63 | if (n == 0.0) {
64 | None
65 | } else {
66 | Some(StandardDeviationState(n, row.getDouble(1), row.getDouble(2)))
67 | }
68 | }
69 | }
70 |
71 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
72 | hasColumn(column) :: isNumeric(column) :: Nil
73 | }
74 |
75 | override def filterCondition: Option[String] = where
76 | }
77 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Sum.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Preconditions.{hasColumn, isNumeric}
20 | import org.apache.spark.sql.functions.sum
21 | import org.apache.spark.sql.types.{DoubleType, StructType}
22 | import org.apache.spark.sql.{Column, Row}
23 | import Analyzers._
24 |
25 | case class SumState(sum: Double) extends DoubleValuedState[SumState] {
26 |
27 | override def sum(other: SumState): SumState = {
28 | SumState(sum + other.sum)
29 | }
30 |
31 | override def metricValue(): Double = {
32 | sum
33 | }
34 | }
35 |
36 | case class Sum(column: String, where: Option[String] = None)
37 | extends StandardScanShareableAnalyzer[SumState]("Sum", column)
38 | with FilterableAnalyzer {
39 |
40 | override def aggregationFunctions(): Seq[Column] = {
41 | sum(conditionalSelection(column, where)).cast(DoubleType) :: Nil
42 | }
43 |
44 | override def fromAggregationResult(result: Row, offset: Int): Option[SumState] = {
45 | ifNoNullsIn(result, offset) { _ =>
46 | SumState(result.getDouble(offset))
47 | }
48 | }
49 |
50 | override protected def additionalPreconditions(): Seq[StructType => Unit] = {
51 | hasColumn(column) :: isNumeric(column) :: Nil
52 | }
53 |
54 | override def filterCondition: Option[String] = where
55 | }
56 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/UniqueValueRatio.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers.COUNT_COL
20 | import com.amazon.deequ.analyzers.FilteredRowOutcome.FilteredRowOutcome
21 | import com.amazon.deequ.metrics.DoubleMetric
22 | import org.apache.spark.sql.functions.expr
23 | import org.apache.spark.sql.functions.not
24 | import org.apache.spark.sql.functions.when
25 | import org.apache.spark.sql.{Column, Row}
26 | import org.apache.spark.sql.functions.{col, count, lit, sum}
27 | import org.apache.spark.sql.types.DoubleType
28 |
29 | case class UniqueValueRatio(columns: Seq[String], where: Option[String] = None,
30 | analyzerOptions: Option[AnalyzerOptions] = None)
31 | extends ScanShareableFrequencyBasedAnalyzer("UniqueValueRatio", columns)
32 | with FilterableAnalyzer {
33 |
34 | override def aggregationFunctions(numRows: Long): Seq[Column] = {
35 | sum(col(COUNT_COL).equalTo(lit(1)).cast(DoubleType)) :: count("*") :: Nil
36 | }
37 |
38 | override def fromAggregationResult(result: Row, offset: Int, fullColumn: Option[Column] = None): DoubleMetric = {
39 | val numUniqueValues = result.getDouble(offset)
40 | val numDistinctValues = result.getLong(offset + 1).toDouble
41 | val conditionColumn = where.map { expression => expr(expression) }
42 | val fullColumnUniqueness = fullColumn.map {
43 | rowLevelColumn => {
44 | conditionColumn.map {
45 | condition => {
46 | when(not(condition), getRowLevelFilterTreatment(analyzerOptions).getExpression)
47 | .when(rowLevelColumn.equalTo(1), true).otherwise(false)
48 | }
49 | }.getOrElse(when(rowLevelColumn.equalTo(1), true).otherwise(false))
50 | }
51 | }
52 | toSuccessMetric(numUniqueValues / numDistinctValues, fullColumnUniqueness)
53 | }
54 |
55 | override def filterCondition: Option[String] = where
56 | }
57 |
58 | object UniqueValueRatio {
59 | def apply(column: String): UniqueValueRatio = {
60 | new UniqueValueRatio(column :: Nil)
61 | }
62 |
63 | def apply(column: String, where: Option[String]): UniqueValueRatio = {
64 | new UniqueValueRatio(column :: Nil, where)
65 | }
66 | }
67 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/Uniqueness.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.analyzers.Analyzers.COUNT_COL
20 | import com.amazon.deequ.analyzers.FilteredRowOutcome.FilteredRowOutcome
21 | import com.amazon.deequ.metrics.DoubleMetric
22 | import com.google.common.annotations.VisibleForTesting
23 | import org.apache.spark.sql.Column
24 | import org.apache.spark.sql.Row
25 | import org.apache.spark.sql.functions.when
26 | import org.apache.spark.sql.functions.col
27 | import org.apache.spark.sql.functions.not
28 | import org.apache.spark.sql.functions.expr
29 | import org.apache.spark.sql.functions.lit
30 | import org.apache.spark.sql.functions.sum
31 | import org.apache.spark.sql.types.DoubleType
32 |
33 | /** Uniqueness is the fraction of unique values of a column(s), i.e.,
34 | * values that occur exactly once. */
35 | case class Uniqueness(columns: Seq[String], where: Option[String] = None,
36 | analyzerOptions: Option[AnalyzerOptions] = None)
37 | extends ScanShareableFrequencyBasedAnalyzer("Uniqueness", columns)
38 | with FilterableAnalyzer {
39 |
40 | override def aggregationFunctions(numRows: Long): Seq[Column] = {
41 | (sum(col(COUNT_COL).equalTo(lit(1)).cast(DoubleType)) / numRows) :: Nil
42 | }
43 |
44 | override def fromAggregationResult(result: Row, offset: Int, fullColumn: Option[Column]): DoubleMetric = {
45 | val conditionColumn = where.map { expression => expr(expression) }
46 | val fullColumnUniqueness = fullColumn.map {
47 | rowLevelColumn => {
48 | conditionColumn.map {
49 | condition => {
50 | when(not(condition), getRowLevelFilterTreatment(analyzerOptions).getExpression)
51 | .when(rowLevelColumn.equalTo(1), true).otherwise(false)
52 | }
53 | }.getOrElse(when(rowLevelColumn.equalTo(1), true).otherwise(false))
54 | }
55 | }
56 | super.fromAggregationResult(result, offset, fullColumnUniqueness)
57 | }
58 |
59 | override def filterCondition: Option[String] = where
60 | }
61 |
62 | object Uniqueness {
63 | def apply(column: String): Uniqueness = {
64 | new Uniqueness(column :: Nil)
65 | }
66 |
67 | def apply(column: String, where: Option[String]): Uniqueness = {
68 | new Uniqueness(column :: Nil, where)
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/catalyst/DeequFunctions.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 |
20 | import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateFunction, StatefulApproxQuantile, StatefulHyperloglogPlus}
21 | import org.apache.spark.sql.catalyst.expressions.Literal
22 |
23 | /* Custom aggregation functions used internally by deequ */
24 | object DeequFunctions {
25 |
26 | private[this] def withAggregateFunction(
27 | func: AggregateFunction,
28 | isDistinct: Boolean = false): Column = {
29 |
30 | Column(func.toAggregateExpression(isDistinct))
31 | }
32 |
33 | /** Pearson correlation with state */
34 | def stateful_corr(columnA: String, columnB: String): Column = {
35 | stateful_corr(Column(columnA), Column(columnB))
36 | }
37 |
38 | /** Pearson correlation with state */
39 | def stateful_corr(columnA: Column, columnB: Column): Column = withAggregateFunction {
40 | new StatefulCorrelation(columnA.expr, columnB.expr)
41 | }
42 |
43 | /** Standard deviation with state */
44 | def stateful_stddev_pop(column: String): Column = {
45 | stateful_stddev_pop(Column(column))
46 | }
47 |
48 | /** Standard deviation with state */
49 | def stateful_stddev_pop(column: Column): Column = withAggregateFunction {
50 | StatefulStdDevPop(column.expr)
51 | }
52 |
53 | /** Approximate number of distinct values with state via HLL's */
54 | def stateful_approx_count_distinct(column: String): Column = {
55 | stateful_approx_count_distinct(Column(column))
56 | }
57 |
58 | /** Approximate number of distinct values with state via HLL's */
59 | def stateful_approx_count_distinct(column: Column): Column = withAggregateFunction {
60 | StatefulHyperloglogPlus(column.expr)
61 | }
62 |
63 | def stateful_approx_quantile(
64 | column: Column,
65 | relativeError: Double)
66 | : Column = withAggregateFunction {
67 |
68 | StatefulApproxQuantile(
69 | column.expr,
70 | // val relativeError = 1.0D / accuracy inside StatefulApproxQuantile
71 | Literal(1.0 / relativeError),
72 | mutableAggBufferOffset = 0,
73 | inputAggBufferOffset = 0
74 | )
75 | }
76 |
77 | /** Data type detection with state */
78 | def stateful_datatype(column: Column): Column = {
79 | val statefulDataType = new StatefulDataType()
80 | statefulDataType(column)
81 | }
82 |
83 | def stateful_kll(
84 | column: Column,
85 | sketchSize: Int,
86 | shrinkingFactor: Double): Column = {
87 | val statefulKLL = new StatefulKLLSketch(sketchSize, shrinkingFactor)
88 | statefulKLL(column)
89 | }
90 | }
91 |
92 |
93 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/catalyst/StatefulCorrelation.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import org.apache.spark.sql.catalyst.expressions.aggregate.Corr
20 | import org.apache.spark.sql.catalyst.expressions._
21 | import org.apache.spark.sql.types._
22 |
23 | /** Adjusted version of org.apache.spark.sql.catalyst.expressions.aggregate.Corr */
24 | private[sql] class StatefulCorrelation(
25 | x: Expression,
26 | y: Expression,
27 | nullOnDivideByZero: Boolean = false
28 | ) extends Corr(x, y, nullOnDivideByZero) {
29 |
30 | override def dataType: org.apache.spark.sql.types.DataType =
31 | StructType(StructField("n", DoubleType) :: StructField("xAvg", DoubleType) ::
32 | StructField("yAvg", DoubleType) :: StructField("ck", DoubleType) ::
33 | StructField("xMk", DoubleType) :: StructField("yMk", DoubleType) :: Nil)
34 |
35 | override val evaluateExpression: Expression = {
36 | CreateStruct(n :: xAvg :: yAvg :: ck :: xMk :: yMk :: Nil)
37 | }
38 |
39 | override def prettyName: String = "stateful_corr"
40 |
41 | override def canEqual(other: Any): Boolean = other.isInstanceOf[StatefulCorrelation]
42 |
43 | override def equals(other: Any): Boolean = other match {
44 | case that: StatefulCorrelation =>
45 | (that canEqual this) && evaluateExpression == that.evaluateExpression
46 | case _ => false
47 | }
48 |
49 | override def hashCode(): Int = {
50 | val state = Seq(super.hashCode(), evaluateExpression)
51 | state.map { _.hashCode() }.foldLeft(0) {(a, b) => 31 * a + b }
52 | }
53 |
54 | override protected def withNewChildrenInternal(newLeft: Expression,
55 | newRight: Expression): StatefulCorrelation =
56 | new StatefulCorrelation(newLeft, newRight, nullOnDivideByZero)
57 | }
58 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/catalyst/StatefulStdDevPop.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package org.apache.spark.sql
18 |
19 | import org.apache.spark.sql.catalyst.expressions._
20 | import org.apache.spark.sql.catalyst.expressions.aggregate.CentralMomentAgg
21 | import org.apache.spark.sql.types._
22 |
23 | /** Adjusted version of org.apache.spark.sql.catalyst.expressions.aggregate.StddevPop */
24 | private[sql] case class StatefulStdDevPop(
25 | child: Expression,
26 | nullOnDivideByZero: Boolean = false
27 | ) extends CentralMomentAgg(child, nullOnDivideByZero) {
28 |
29 | override protected def momentOrder = 2
30 |
31 | override def dataType: DataType = StructType(StructField("n", DoubleType) ::
32 | StructField("avg", DoubleType) :: StructField("m2", DoubleType) :: Nil)
33 |
34 | override val evaluateExpression: Expression = CreateStruct(n :: avg :: m2 :: Nil)
35 |
36 | override def prettyName: String = "stateful_stddev_pop"
37 |
38 | protected def withNewChildInternal(newChild: Expression): StatefulStdDevPop =
39 | copy(child = newChild)
40 | }
41 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/analyzers/runners/MetricCalculationException.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers.runners
18 |
19 | abstract class MetricCalculationException(message: String) extends Exception(message)
20 |
21 | class MetricCalculationRuntimeException(message: String)
22 | extends MetricCalculationException(message) {
23 |
24 | def this(message: String, cause: Throwable) {
25 | this(message)
26 | initCause(cause)
27 | }
28 |
29 | def this(cause: Throwable) {
30 | this(Option(cause).map(_.toString).orNull, cause)
31 | }
32 | }
33 |
34 | class MetricCalculationPreconditionException(message: String)
35 | extends MetricCalculationException(message)
36 |
37 |
38 | class NoSuchColumnException(message: String)
39 | extends MetricCalculationPreconditionException(message)
40 |
41 | class WrongColumnTypeException(message: String)
42 | extends MetricCalculationPreconditionException(message)
43 |
44 | class NoColumnsSpecifiedException(message: String)
45 | extends MetricCalculationPreconditionException(message)
46 |
47 | class NumberOfSpecifiedColumnsException(message: String)
48 | extends MetricCalculationPreconditionException(message)
49 |
50 | class IllegalAnalyzerParameterException(
51 | message: String)
52 | extends MetricCalculationPreconditionException(message)
53 |
54 | class EmptyStateException(message: String) extends MetricCalculationRuntimeException(message)
55 |
56 |
57 | object MetricCalculationException {
58 |
59 | private[deequ] def getApproxQuantileIllegalParamMessage(quantile: Double): String = {
60 | "Quantile parameter must be in the closed interval [0, 1]. " +
61 | s"Currently, the value is: $quantile!"
62 | }
63 |
64 | private[deequ] def getApproxQuantileIllegalErrorParamMessage(relativeError: Double): String = {
65 | "Relative error parameter must be in the closed interval [0, 1]. " +
66 | s"Currently, the value is: $relativeError!"
67 | }
68 |
69 | def wrapIfNecessary(exception: Throwable)
70 | : MetricCalculationException = {
71 |
72 | exception match {
73 | case error: MetricCalculationException => error
74 | case error: Throwable => new MetricCalculationRuntimeException(error)
75 | }
76 | }
77 |
78 | }
79 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/anomalydetection/AbsoluteChangeStrategy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | /**
20 | * Detects anomalies based on the values' absolute change.
21 | * The order of the difference can be set manually.
22 | * If it is set to 0, this strategy acts like the [[SimpleThresholdStrategy]].
23 | *
24 | * AbsoluteChangeStrategy(Some(-10.0), Some(10.0), 1) for example
25 | * calculates the first discrete difference
26 | * and if some point's value changes by more than 10.0 in one timestep, it flags it as an anomaly.
27 | *
28 | * @param maxRateDecrease Upper bound of accepted decrease (lower bound of increase).
29 | * @param maxRateIncrease Upper bound of accepted growth.
30 | * @param order Order of the calculated difference.
31 | * Set to 1 it calculates the difference between two consecutive values.
32 | */
33 | case class AbsoluteChangeStrategy(
34 | maxRateDecrease: Option[Double] = None,
35 | maxRateIncrease: Option[Double] = None,
36 | order: Int = 1) extends BaseChangeStrategy
37 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/anomalydetection/AnomalyDetectionStrategy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | /** Interface for all strategies that spot anomalies in a series of data points. */
20 | trait AnomalyDetectionStrategy {
21 |
22 | /**
23 | * Search for anomalies in a series of data points.
24 | *
25 | * @param dataSeries The data contained in a Vector of Doubles
26 | * @param searchInterval The indices between which anomalies should be detected. [a, b).
27 | * @return The indices of all anomalies in the interval and their corresponding wrapper object.
28 | */
29 | def detect(
30 | dataSeries: Vector[Double],
31 | searchInterval: (Int, Int) = (0, Int.MaxValue)): Seq[(Int, Anomaly)]
32 | }
33 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/anomalydetection/DetectionResult.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | class Anomaly(
20 | val value: Option[Double],
21 | val confidence: Double,
22 | val detail: Option[String] = None) {
23 |
24 | def canEqual(that: Any): Boolean = {
25 | that.isInstanceOf[Anomaly]
26 | }
27 |
28 | /**
29 | * Tests anomalies for equality. Ignores detailed explanation.
30 | *
31 | * @param obj The object/ anomaly to compare against
32 | * @return true, if and only if the value and confidence are the same
33 | */
34 | override def equals(obj: Any): Boolean = {
35 | obj match {
36 | case anomaly: Anomaly => anomaly.value == value && anomaly.confidence == confidence
37 | case _ => false
38 | }
39 | }
40 |
41 | override def hashCode: Int = {
42 | val prime = 31
43 | var result = 1
44 | result = prime * result + (if (value == null) 0 else value.hashCode)
45 | prime * result + confidence.hashCode
46 | }
47 |
48 | }
49 |
50 | object Anomaly {
51 | def apply(value: Option[Double], confidence: Double, detail: Option[String] = None): Anomaly = {
52 | new Anomaly(value, confidence, detail)
53 | }
54 | }
55 |
56 | case class DetectionResult(anomalies: Seq[(Long, Anomaly)] = Seq.empty)
57 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/anomalydetection/HistoryUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | import com.amazon.deequ.metrics.Metric
20 |
21 | /**
22 | * Contains utility methods to convert tuples of date and metric to a DataPoint
23 | */
24 | private[deequ] object HistoryUtils {
25 |
26 | /**
27 | * Given a sequence of dated optional metrics, return sequence of dated optional metric values.
28 | *
29 | * @param metrics Sequence of dated optional metrics
30 | * @tparam M Type of the metric value
31 | * @return Sequence of dated optional metric values
32 | */
33 | def extractMetricValues[M](metrics: Seq[(Long, Option[Metric[M]])]): Seq[DataPoint[M]] = {
34 | metrics.map { case (date, metric) => DataPoint(date, extractMetricValue[M](metric)) }
35 | }
36 |
37 | /**
38 | * Given an optional metric,returns optional metric value
39 | *
40 | * @param metric Optional metric
41 | * @tparam M Type of the metric value
42 | * @return Optional metric value
43 | */
44 | def extractMetricValue[M](metric: Option[Metric[M]]): Option[M] = {
45 | metric.flatMap(_.value.toOption)
46 | }
47 |
48 | }
49 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/anomalydetection/RateOfChangeStrategy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 |
18 | package com.amazon.deequ.anomalydetection
19 |
20 | /**
21 | * Provided for backwards compatibility.
22 | * the old [[RateOfChangeStrategy]] actually detects absolute changes
23 | * so it has been migrated to [[AbsoluteChangeStrategy]]
24 | * use [[RelativeRateOfChangeStrategy]] if you want to
25 | * detect relative changes to the previous values
26 | */
27 | @deprecated("use AbsoluteChangeStrategy instead which describes the strategy more accurately")
28 | case class RateOfChangeStrategy(
29 | maxRateDecrease: Option[Double] = None,
30 | maxRateIncrease: Option[Double] = None,
31 | order: Int = 1) extends BaseChangeStrategy
32 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/anomalydetection/RelativeRateOfChangeStrategy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | import breeze.linalg.DenseVector
20 |
21 | /**
22 | * Detects anomalies based on the values' rate of change.
23 | * The order of the difference can be set manually.
24 | * If it is set to 0, this strategy acts like the [[SimpleThresholdStrategy]].
25 | *
26 | * RelativeRateOfChangeStrategy(Some(0.9), Some(1.1), 1) for example
27 | * calculates the first discrete difference
28 | * and if some point's value changes by more than 10.0 Percent in one timestep,
29 | * it flags it as an anomaly.
30 | *
31 | * @param maxRateDecrease Lower bound of accepted relative change (as new value / old value).
32 | * @param maxRateIncrease Upper bound of accepted relative change (as new value / old value).
33 | * @param order Order of the calculated difference.
34 | * Set to 1 it calculates the difference between two consecutive values.
35 | */
36 | case class RelativeRateOfChangeStrategy(
37 | maxRateDecrease: Option[Double] = None,
38 | maxRateIncrease: Option[Double] = None,
39 | order: Int = 1)
40 | extends BaseChangeStrategy {
41 |
42 | /**
43 | * Calculates the rate of change with respect to the specified order.
44 | * If the order is set to 1, the resulting value for a point at index i
45 | * is equal to dataSeries (i) / dataSeries(i - 1).
46 | * Note that this difference cannot be calculated for the first [[order]] elements in the vector.
47 | * The resulting vector is therefore smaller by [[order]] elements.
48 | *
49 | * @param dataSeries The values contained in a DenseVector[Double]
50 | * @param order The order of the derivative.
51 | * @return A vector with the resulting rates of change for all values
52 | * except the first [[order]] elements.
53 | */
54 | override def diff(dataSeries: DenseVector[Double], order: Int): DenseVector[Double] = {
55 | require(order > 0, "Order of diff cannot be zero or negative")
56 | if (dataSeries.length == 0) {
57 | dataSeries
58 | } else {
59 | val valuesRight = dataSeries.slice(order, dataSeries.length)
60 | val valuesLeft = dataSeries.slice(0, dataSeries.length - order)
61 | valuesRight / valuesLeft
62 | }
63 | }
64 | }
65 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/anomalydetection/SimpleThresholdStrategy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | /**
20 | * A simple anomaly detection method that checks if values are in a specified range.
21 | *
22 | * @param lowerBound Lower bound of accepted range of values
23 | * @param upperBound Upper bound of accepted range of values
24 | */
25 | case class SimpleThresholdStrategy(
26 | lowerBound: Double = Double.MinValue,
27 | upperBound: Double)
28 | extends AnomalyDetectionStrategy {
29 |
30 | require(lowerBound <= upperBound, "The lower bound must be smaller or equal to the upper bound.")
31 |
32 | /**
33 | * Search for anomalies in a series of data points.
34 | *
35 | * @param dataSeries The data contained in a Vector of Doubles
36 | * @param searchInterval The indices between which anomalies should be detected. [a, b).
37 | * @return The indices of all anomalies in the interval and their corresponding wrapper object.
38 | */
39 | override def detect(
40 | dataSeries: Vector[Double],
41 | searchInterval: (Int, Int)): Seq[(Int, Anomaly)] = {
42 |
43 | val (searchStart, searchEnd) = searchInterval
44 |
45 | require (searchStart <= searchEnd, "The start of the interval can't be larger than the end.")
46 |
47 | dataSeries.zipWithIndex
48 | .slice(searchStart, searchEnd)
49 | .filter { case (value, _) => value < lowerBound || value > upperBound }
50 | .map { case (value, index) =>
51 |
52 | val detail = Some(s"[SimpleThresholdStrategy]: Value $value is not in " +
53 | s"bounds [$lowerBound, $upperBound]")
54 |
55 | (index, Anomaly(Option(value), 1.0, detail))
56 | }
57 | }
58 | }
59 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/checks/CheckWithLastConstraintFilterable.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.checks
18 |
19 | import com.amazon.deequ.constraints.Constraint
20 |
21 | /** Allows to replace the last configured constraint in a check with a filtered version */
22 | class CheckWithLastConstraintFilterable(
23 | level: CheckLevel.Value,
24 | description: String,
25 | constraints: Seq[Constraint],
26 | createReplacement: Option[String] => Constraint)
27 | extends Check(level, description, constraints) {
28 |
29 | /**
30 | * Defines a filter to apply before evaluating the previous constraint
31 | *
32 | * @param filter SparkSQL predicate to apply
33 | * @return
34 | */
35 | def where(filter: String): Check = {
36 |
37 | val adjustedConstraints =
38 | constraints.take(constraints.size - 1) :+ createReplacement(Option(filter))
39 |
40 | Check(level, description, adjustedConstraints)
41 | }
42 | }
43 |
44 | object CheckWithLastConstraintFilterable {
45 | def apply(
46 | level: CheckLevel.Value,
47 | description: String,
48 | constraints: Seq[Constraint],
49 | createReplacement: Option[String] => Constraint
50 | ): CheckWithLastConstraintFilterable = {
51 |
52 | new CheckWithLastConstraintFilterable(level, description, constraints, createReplacement)
53 | }
54 | }
55 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/checks/ColumnCondition.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.checks
18 |
19 | import org.apache.spark.sql.functions.{col}
20 |
21 | private[checks] object ColumnCondition {
22 |
23 | def isEachNotNull(cols: Seq[String]): String = {
24 | cols
25 | .map(col(_).isNotNull)
26 | .reduce(_ and _)
27 | .toString()
28 | }
29 |
30 | def isAnyNotNull(cols: Seq[String]): String = {
31 | cols
32 | .map(col(_).isNotNull)
33 | .reduce(_ or _)
34 | .toString()
35 | }
36 | }
37 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/comparison/ComparisonBase.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2022 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.comparison
18 |
19 | trait ComparisonBase {
20 | val defaultOutcomeColumnName = "outcome"
21 | val referenceColumnNamePrefix = "ref_col"
22 | }
23 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/comparison/ComparisonResult.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.comparison
18 |
19 | sealed trait ComparisonResult
20 |
21 | case class ComparisonFailed(errorMessage: String, ratio: Double = 0) extends ComparisonResult
22 | case class ComparisonSucceeded(ratio: Double = 0) extends ComparisonResult
23 |
24 | case class DatasetMatchFailed(errorMessage: String, passedCount: Option[Long] = None,
25 | totalCount: Option[Long] = None) extends ComparisonResult
26 | case class DatasetMatchSucceeded(passedCount: Long, totalCount: Long) extends ComparisonResult
27 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/constraints/ConstrainableDataTypes.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.constraints
18 |
19 | object ConstrainableDataTypes extends Enumeration {
20 | val Null: Value = Value(0)
21 | val Fractional: Value = Value(1)
22 | val Integral: Value = Value(2)
23 | val Boolean: Value = Value(3)
24 | val String: Value = Value(4)
25 | val Numeric: Value = Value(5) // Union of integral and fractional
26 | }
27 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/dqdl/EvaluateDataQuality.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl
18 |
19 | import com.amazon.deequ.dqdl.execution.DQDLExecutor
20 | import com.amazon.deequ.dqdl.model.ExecutableRule
21 | import com.amazon.deequ.dqdl.translation.{DQDLRuleTranslator, DeequOutcomeTranslator}
22 | import com.amazon.deequ.dqdl.util.DefaultDQDLParser
23 | import org.apache.spark.sql.DataFrame
24 | import software.amazon.glue.dqdl.model.DQRuleset
25 |
26 | /**
27 | * Entry point for evaluating data quality.
28 | *
29 | * Given a Spark DataFrame and a DQDL ruleset (as a String), this object:
30 | * - Parses and translates each rule from the ruleset to an ExecutableRule
31 | * - Executes the rules on the DataFrame.
32 | * - Translates the outcome back to a Spark DataFrame.
33 | */
34 | object EvaluateDataQuality {
35 |
36 | /**
37 | * Validates the given Spark DataFrame against a set of data quality rules defined in DQDL format.
38 | *
39 | *
This method applies the specified data quality ruleset to the input DataFrame and returns a new
40 | * DataFrame summarizing the overall quality results, including any issues detected during the analysis.
41 | *
42 | * @param df the Spark DataFrame to analyze.
43 | * @param rulesetDefinition the data quality ruleset (defined in DQDL string format) to apply to the DataFrame.
44 | * @return a Spark DataFrame containing the aggregated data quality results.
45 | */
46 | def process(df: DataFrame, rulesetDefinition: String): DataFrame = {
47 | // 1. Parse the ruleset
48 | val ruleset: DQRuleset = DefaultDQDLParser.parse(rulesetDefinition)
49 |
50 | // 2. Translate the dqRuleset into a corresponding ExecutableRules.
51 | val executableRules: Seq[ExecutableRule] = DQDLRuleTranslator.toExecutableRules(ruleset)
52 |
53 | // 3. Execute the rules against the DataFrame.
54 | val result = DQDLExecutor.executeRules(df, Seq.empty)
55 |
56 | // 4. Translate the results into a Spark DataFrame.
57 | val outcomeTranslator = new DeequOutcomeTranslator(df.sparkSession)
58 | outcomeTranslator.translate(result)
59 | }
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/dqdl/execution/DQDLExecutor.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl.execution
18 |
19 | import com.amazon.deequ.checks.Check
20 | import com.amazon.deequ.{VerificationResult, VerificationSuite}
21 | import org.apache.spark.sql.DataFrame
22 |
23 | /**
24 | * Executes a sequence of Deequ Checks on a Spark DataFrame.
25 | */
26 | object DQDLExecutor {
27 | def executeRules(df: DataFrame, checks: Seq[Check]): VerificationResult = {
28 | VerificationSuite()
29 | .onData(df)
30 | .addChecks(checks)
31 | .run()
32 | }
33 | }
34 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/dqdl/model/ExecutableRule.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl.model
18 |
19 | import com.amazon.deequ.checks.Check
20 | import com.amazon.deequ.dqdl.util.DQDLUtility.convertWhereClauseForMetric
21 | import software.amazon.glue.dqdl.model.DQRule
22 |
23 | trait ExecutableRule {
24 | val dqRule: DQRule
25 | val evaluatedMetricName: Option[String]
26 | }
27 |
28 | case class UnsupportedExecutableRule(dqRule: DQRule, reason: Option[String] = None) extends ExecutableRule {
29 | override val evaluatedMetricName: Option[String] = None
30 | }
31 |
32 | case class DeequExecutableRule(dqRule: DQRule,
33 | check: Check,
34 | deequMetricMappings: Seq[DeequMetricMapping] = Seq.empty) extends ExecutableRule {
35 |
36 | private val Delim = "."
37 | val evaluatedMetricName: Option[String] = deequMetricMappings match {
38 | case mappings if mappings.size == 1 =>
39 | Some(s"${mappings.head.entity}$Delim${mappings.head.instance}$Delim${mappings.head.name}")
40 | case _ => None // Multiple metrics reported for rule; cannot determine name for general case
41 | }
42 | }
43 |
44 | case class DeequMetricMapping(entity: String,
45 | instance: String,
46 | name: String,
47 | deequName: String,
48 | deequInstance: Option[String] = None,
49 | disambiguator: Option[String] = None)
50 |
51 | object DeequMetricMapping {
52 | def apply(entity: String,
53 | instance: String,
54 | name: String,
55 | deequName: String,
56 | deequInstance: Option[String],
57 | rule: DQRule): DeequMetricMapping = {
58 | new DeequMetricMapping(
59 | entity,
60 | instance,
61 | name,
62 | deequName,
63 | deequInstance,
64 | convertWhereClauseForMetric(rule.getWhereClause)
65 | )
66 | }
67 | }
68 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/dqdl/translation/DQDLRuleConverter.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl.translation
18 |
19 | import com.amazon.deequ.checks.{Check, CheckLevel}
20 | import com.amazon.deequ.dqdl.execution.DefaultOperandEvaluator
21 | import com.amazon.deequ.dqdl.model.DeequMetricMapping
22 | import com.amazon.deequ.dqdl.util.DQDLUtility.addWhereClause
23 | import software.amazon.glue.dqdl.model.DQRule
24 | import software.amazon.glue.dqdl.model.condition.number.NumberBasedCondition
25 |
26 |
27 | trait DQDLRuleConverter {
28 | def translate(rule: DQRule): Either[String, (Check, Seq[DeequMetricMapping])]
29 |
30 | def assertionAsScala(dqRule: DQRule, e: NumberBasedCondition): Double => Boolean = {
31 | val evaluator = DefaultOperandEvaluator
32 | (d: Double) => e.evaluate(d, dqRule, evaluator)
33 | }
34 | }
35 |
36 | case class RowCountRuleConverter() extends DQDLRuleConverter {
37 | override def translate(rule: DQRule): Either[String, (Check, Seq[DeequMetricMapping])] = {
38 | val fn = assertionAsScala(rule, rule.getCondition.asInstanceOf[NumberBasedCondition])
39 | val check = Check(CheckLevel.Error, java.util.UUID.randomUUID.toString).hasSize(rc => fn(rc.toDouble))
40 | Right(addWhereClause(rule, check), Seq(DeequMetricMapping("Dataset", "*", "RowCount", "Size", None, rule = rule)))
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/dqdl/translation/DQDLRuleTranslator.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl.translation
18 |
19 | import com.amazon.deequ.dqdl.model.{DeequExecutableRule, ExecutableRule, UnsupportedExecutableRule}
20 | import software.amazon.glue.dqdl.model.{DQRule, DQRuleset}
21 |
22 | import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter
23 |
24 |
25 | /**
26 | * Translates DQDL rules into ExecutableRules.
27 | * Allows registration of specific converters for different rule types.
28 | */
29 | object DQDLRuleTranslator {
30 |
31 | // Map from rule type to its converter implementation.
32 | private var converters: Map[String, DQDLRuleConverter] = Map.empty
33 |
34 | register("RowCount", new RowCountRuleConverter)
35 |
36 | private def register(ruleType: String, converter: DQDLRuleConverter): Unit = {
37 | converters += (ruleType -> converter)
38 | }
39 |
40 | /**
41 | * Translates a single DQDL rule
42 | */
43 | private[dqdl] def translateRule(rule: DQRule): Either[String, DeequExecutableRule] = {
44 | converters.get(rule.getRuleType) match {
45 | case None =>
46 | Left(s"No converter found for rule type: ${rule.getRuleType}")
47 | case Some(converter) =>
48 | converter.translate(rule) map { case (check, deequMetrics) => DeequExecutableRule(rule, check, deequMetrics) }
49 | }
50 | }
51 |
52 | private[dqdl] def toExecutableRule(rule: DQRule): ExecutableRule = {
53 | translateRule(rule) match {
54 | case Right(deequExecutableRule) => deequExecutableRule
55 | case Left(message) => UnsupportedExecutableRule(rule, Some(message))
56 | }
57 | }
58 |
59 | /**
60 | * Translate a ruleset to executable rules
61 | */
62 | def toExecutableRules(ruleset: DQRuleset): Seq[ExecutableRule] = {
63 | ruleset.getRules.asScala.map(toExecutableRule).toSeq.distinct
64 | }
65 |
66 | }
67 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/dqdl/translation/DeequOutcomeTranslator.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl.translation
18 |
19 | import com.amazon.deequ.VerificationResult
20 | import org.apache.spark.sql.{DataFrame, SparkSession}
21 |
22 | /**
23 | * Translates the outcome of Deequ checks (VerificationResult)
24 | * into a Spark DataFrame containing the results.
25 | */
26 | class DeequOutcomeTranslator(spark: SparkSession) {
27 | def translate(result: VerificationResult): DataFrame = {
28 | // todo implement
29 | spark.emptyDataFrame
30 | }
31 | }
32 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/dqdl/util/DQDLUtility.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl.util
18 |
19 | import com.amazon.deequ.checks.{Check, CheckWithLastConstraintFilterable}
20 | import software.amazon.glue.dqdl.model.DQRule
21 |
22 | object DQDLUtility {
23 |
24 | def convertWhereClauseForMetric(whereClause: String): Option[String] =
25 | Option(whereClause).map(_ => s"(where: $whereClause)")
26 |
27 | def isWhereClausePresent(rule: DQRule): Boolean = {
28 | rule.getWhereClause != null
29 | }
30 |
31 | def addWhereClause(rule: DQRule, check: CheckWithLastConstraintFilterable): Check =
32 | if (isWhereClausePresent(rule)) check.where(rule.getWhereClause)
33 | else check
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/dqdl/util/DefaultDQDLParser.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl.util
18 |
19 | import software.amazon.glue.dqdl.exception.InvalidDataQualityRulesetException
20 | import software.amazon.glue.dqdl.model.DQRuleset
21 | import software.amazon.glue.dqdl.parser.DQDLParser
22 |
23 | import scala.util.{Failure, Success, Try}
24 |
25 | trait DQDLParserTrait {
26 | def parse(ruleset: String): DQRuleset
27 | }
28 |
29 | object DefaultDQDLParser extends DQDLParserTrait {
30 | override def parse(ruleset: String): DQRuleset = {
31 | val dqdlParser: DQDLParser = new DQDLParser()
32 | val dqRuleset: DQRuleset = Try {
33 | dqdlParser.parse(ruleset)
34 | } match {
35 | case Success(value) => value
36 | case Failure(ex: InvalidDataQualityRulesetException) => throw new IllegalArgumentException(ex.getMessage)
37 | case Failure(ex) => throw ex
38 | }
39 | dqRuleset
40 | }
41 | }
42 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/examples/BasicExample.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.examples
18 |
19 | import ExampleUtils.{withSpark, itemsAsDataframe}
20 | import com.amazon.deequ.VerificationSuite
21 | import com.amazon.deequ.checks.{Check, CheckLevel, CheckStatus}
22 | import com.amazon.deequ.constraints.ConstraintStatus
23 |
24 | private[examples] object BasicExample extends App {
25 |
26 | withSpark { session =>
27 |
28 | val data = itemsAsDataframe(session,
29 | Item(1, "Thingy A", "awesome thing.", "high", 0),
30 | Item(2, "Thingy B", "available at http://thingb.com", null, 0),
31 | Item(3, null, null, "low", 5),
32 | Item(4, "Thingy D", "checkout https://thingd.ca", "low", 10),
33 | Item(5, "Thingy E", null, "high", 12))
34 |
35 | val verificationResult = VerificationSuite()
36 | .onData(data)
37 | .addCheck(
38 | Check(CheckLevel.Error, "integrity checks")
39 | // we expect 5 records
40 | .hasSize(_ == 5)
41 | // 'id' should never be NULL
42 | .isComplete("id")
43 | // 'id' should not contain duplicates
44 | .isUnique("id")
45 | // 'productName' should never be NULL
46 | .isComplete("productName")
47 | // 'priority' should only contain the values "high" and "low"
48 | .isContainedIn("priority", Array("high", "low"))
49 | // 'numViews' should not contain negative values
50 | .isNonNegative("numViews"))
51 | .addCheck(
52 | Check(CheckLevel.Warning, "distribution checks")
53 | // at least half of the 'description's should contain a url
54 | .containsURL("description", _ >= 0.5)
55 | // half of the items should have less than 10 'numViews'
56 | .hasApproxQuantile("numViews", 0.5, _ <= 10))
57 | .run()
58 |
59 | if (verificationResult.status == CheckStatus.Success) {
60 | println("The data passed the test, everything is fine!")
61 | } else {
62 | println("We found errors in the data, the following constraints were not satisfied:\n")
63 |
64 | val resultsForAllConstraints = verificationResult.checkResults
65 | .flatMap { case (_, checkResult) => checkResult.constraintResults }
66 |
67 | resultsForAllConstraints
68 | .filter { _.status != ConstraintStatus.Success }
69 | .foreach { result =>
70 | println(s"${result.constraint} failed: ${result.message.get}")
71 | }
72 | }
73 |
74 | }
75 | }
76 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/examples/ExampleUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.examples
18 |
19 | import org.apache.spark.sql.{DataFrame, SparkSession}
20 |
21 | private[deequ] object ExampleUtils {
22 |
23 | def withSpark(func: SparkSession => Unit): Unit = {
24 | val session = SparkSession.builder()
25 | .master("local")
26 | .appName("test")
27 | .config("spark.ui.enabled", "false")
28 | .getOrCreate()
29 | session.sparkContext.setCheckpointDir(System.getProperty("java.io.tmpdir"))
30 |
31 | try {
32 | func(session)
33 | } finally {
34 | session.stop()
35 | System.clearProperty("spark.driver.port")
36 | }
37 | }
38 |
39 | def itemsAsDataframe(session: SparkSession, items: Item*): DataFrame = {
40 | val rdd = session.sparkContext.parallelize(items)
41 | session.createDataFrame(rdd)
42 | }
43 |
44 | def manufacturersAsDataframe(session: SparkSession, manufacturers: Manufacturer*): DataFrame = {
45 | val rdd = session.sparkContext.parallelize(manufacturers)
46 | session.createDataFrame(rdd)
47 | }
48 | }
49 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/examples/IncrementalMetricsExample.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.examples
18 |
19 | import ExampleUtils.{itemsAsDataframe, withSpark}
20 | import com.amazon.deequ.analyzers.{Analysis, ApproxCountDistinct, Completeness, InMemoryStateProvider, Size}
21 | import com.amazon.deequ.analyzers.runners.AnalysisRunner
22 |
23 | private[examples] object IncrementalMetricsExample extends App {
24 |
25 | /* NOTE: Stateful support is still work in progress, and is therefore not yet integrated into
26 | VerificationSuite. We showcase however how to incrementally compute metrics on a growing
27 | dataset using the AnalysisRunner. */
28 |
29 | withSpark { session =>
30 |
31 | val data = itemsAsDataframe(session,
32 | Item(1, "Thingy A", "awesome thing.", "high", 0),
33 | Item(2, "Thingy B", "available tomorrow", "low", 0),
34 | Item(3, "Thing C", null, null, 5))
35 |
36 | val moreData = itemsAsDataframe(session,
37 | Item(4, "Thingy D", null, "low", 10),
38 | Item(5, "Thingy E", null, "high", 12))
39 |
40 |
41 | val analysis = Analysis()
42 | .addAnalyzer(Size())
43 | .addAnalyzer(ApproxCountDistinct("id"))
44 | .addAnalyzer(Completeness("productName"))
45 | .addAnalyzer(Completeness("description"))
46 |
47 | val stateStore = InMemoryStateProvider()
48 |
49 | val metricsForData = AnalysisRunner.run(
50 | data = data,
51 | analysis = analysis,
52 | saveStatesWith = Some(stateStore) // persist the internal state of the computation
53 | )
54 |
55 | // We update the metrics now from the stored states without having to access the previous data!
56 | val metricsAfterAddingMoreData = AnalysisRunner.run(
57 | data = moreData,
58 | analysis = analysis,
59 | aggregateWith = Some(stateStore) // continue from internal state of the computation
60 | )
61 |
62 | println("Metrics for the first 3 records:\n")
63 | metricsForData.metricMap.foreach { case (analyzer, metric) =>
64 | println(s"\t$analyzer: ${metric.value.get}")
65 | }
66 |
67 | println("\nMetrics after adding 2 more records:\n")
68 | metricsAfterAddingMoreData.metricMap.foreach { case (analyzer, metric) =>
69 | println(s"\t$analyzer: ${metric.value.get}")
70 | }
71 |
72 | }
73 | }
74 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/examples/KLLCheckExample.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.examples
18 |
19 | import ExampleUtils.{itemsAsDataframe, withSpark}
20 | import com.amazon.deequ.VerificationSuite
21 | import com.amazon.deequ.analyzers.KLLParameters
22 | import com.amazon.deequ.checks.{Check, CheckLevel, CheckStatus}
23 | import com.amazon.deequ.constraints.ConstraintStatus
24 | import org.apache.spark.sql.types.DoubleType
25 |
26 | private[examples] object KLLCheckExample extends App {
27 |
28 | withSpark { session =>
29 |
30 | val data = itemsAsDataframe(session,
31 | Item(1, "Thingy A", "awesome thing.", "high", 0),
32 | Item(2, "Thingy B", "available at http://thingb.com", null, 0),
33 | Item(3, null, null, "low", 5),
34 | Item(4, "Thingy D", "checkout https://thingd.ca", "low", 10),
35 | Item(5, "Thingy E", null, "high", 12))
36 |
37 | val newData = data.select(data("numViews").cast(DoubleType).as("numViews"))
38 |
39 | val verificationResult = VerificationSuite()
40 | .onData(newData)
41 | .addCheck(
42 | Check(CheckLevel.Error, "integrity checks")
43 | // we expect 5 records
44 | .hasSize(_ == 5)
45 | // we expect the maximum of tips to be not more than 10
46 | .hasMax("numViews", _ <= 10)
47 | // we expect the sketch size to be at least 16
48 | .kllSketchSatisfies("numViews", _.parameters(1) >= 16,
49 | kllParameters = Option(KLLParameters(2, 0.64, 2))))
50 | .run()
51 |
52 | if (verificationResult.status == CheckStatus.Success) {
53 | println("The data passed the test, everything is fine!")
54 | } else {
55 | println("We found errors in the data, the following constraints were not satisfied:\n")
56 |
57 | val resultsForAllConstraints = verificationResult.checkResults
58 | .flatMap { case (_, checkResult) => checkResult.constraintResults }
59 |
60 | resultsForAllConstraints
61 | .filter { _.status != ConstraintStatus.Success }
62 | .foreach { result =>
63 | println(s"${result.constraint} failed: ${result.message.get}")
64 | }
65 | }
66 |
67 | }
68 | }
69 |
70 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/examples/entities.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.examples
18 |
19 | private[deequ] case class Item(
20 | id: Long,
21 | productName: String,
22 | description: String,
23 | priority: String,
24 | numViews: Long
25 | )
26 |
27 | private[deequ] case class Manufacturer(
28 | id: Long,
29 | manufacturerName: String,
30 | countryCode: String
31 | )
32 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/io/DfsUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.io
18 |
19 | import java.io.{BufferedWriter, OutputStreamWriter}
20 |
21 | import org.apache.hadoop.fs.{FSDataInputStream, FSDataOutputStream, FileSystem, Path}
22 | import org.apache.spark.sql.SparkSession
23 |
24 | private[deequ] object DfsUtils {
25 |
26 | /* Helper function to read from a binary file on S3 */
27 | def readFromFileOnDfs[T](session: SparkSession, path: String)
28 | (readFunc: FSDataInputStream => T): T = {
29 |
30 | val (fs, qualifiedPath) = asQualifiedPath(session, path)
31 | val input = fs.open(qualifiedPath)
32 |
33 | try {
34 | readFunc(input)
35 | } finally {
36 | if (input != null) {
37 | input.close()
38 | }
39 | }
40 | }
41 |
42 | /* Helper function to write to a binary file on S3 */
43 | def writeToFileOnDfs(session: SparkSession, path: String, overwrite: Boolean = false)
44 | (writeFunc: FSDataOutputStream => Unit): Unit = {
45 |
46 | val (fs, qualifiedPath) = asQualifiedPath(session, path)
47 | val output = fs.create(qualifiedPath, overwrite)
48 |
49 | try {
50 | writeFunc(output)
51 | } finally {
52 | if (output != null) {
53 | output.close()
54 | }
55 | }
56 | }
57 |
58 | /* Helper function to write to a binary file on S3 */
59 | def writeToTextFileOnDfs(session: SparkSession, path: String, overwrite: Boolean = false)
60 | (writeFunc: BufferedWriter => Unit): Unit = {
61 |
62 | val (fs, qualifiedPath) = asQualifiedPath(session, path)
63 | val output = fs.create(qualifiedPath, overwrite)
64 |
65 | try {
66 | val writer = new BufferedWriter(new OutputStreamWriter(output))
67 | writeFunc(writer)
68 | writer.close()
69 | } finally {
70 | if (output != null) {
71 | output.close()
72 | }
73 | }
74 | }
75 |
76 | /* Make sure we write to the correct filesystem, as EMR clusters also have an internal HDFS */
77 | def asQualifiedPath(session: SparkSession, path: String): (FileSystem, Path) = {
78 | val hdfsPath = new Path(path)
79 | val fs = hdfsPath.getFileSystem(session.sparkContext.hadoopConfiguration)
80 | val qualifiedPath = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory)
81 |
82 | (fs, qualifiedPath)
83 | }
84 |
85 | }
86 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/metrics/HistogramMetric.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.metrics
18 |
19 | import scala.util.{Failure, Success, Try}
20 |
21 | case class DistributionValue(absolute: Long, ratio: Double)
22 |
23 | case class Distribution(values: Map[String, DistributionValue], numberOfBins: Long) {
24 |
25 | def apply(key: String): DistributionValue = {
26 | values(key)
27 | }
28 |
29 | def argmax: String = {
30 | val (distributionKey, _) = values.toSeq
31 | .maxBy { case (_, distributionValue) => distributionValue.absolute }
32 |
33 | distributionKey
34 | }
35 | }
36 |
37 | case class HistogramMetric(column: String, value: Try[Distribution]) extends Metric[Distribution] {
38 | val entity: Entity.Value = Entity.Column
39 | val instance: String = column
40 | val name = "Histogram"
41 |
42 | def flatten(): Seq[DoubleMetric] = {
43 | value
44 | .map { distribution =>
45 | val numberOfBins = Seq(DoubleMetric(entity, s"$name.bins", instance,
46 | Success(distribution.numberOfBins.toDouble)))
47 |
48 | val details = distribution.values
49 | .flatMap { case (key, distValue) =>
50 | DoubleMetric(entity, s"$name.abs.$key", instance, Success(distValue.absolute)) ::
51 | DoubleMetric(entity, s"$name.ratio.$key", instance, Success(distValue.ratio)) :: Nil
52 | }
53 | numberOfBins ++ details
54 | }
55 | .recover {
56 | case e: Exception => Seq(DoubleMetric(entity, s"$name.bins", instance, Failure(e)))
57 | }
58 | .get
59 | }
60 |
61 | }
62 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/repository/MetricsRepository.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.repository
18 |
19 | import com.amazon.deequ.analyzers.runners.AnalyzerContext
20 |
21 | /**
22 | * Common trait for RepositoryIndexes where deequ runs can be stored.
23 | * Repository provides methods to store AnalysisResults(metrics) and VerificationResults(if any)
24 | */
25 | trait MetricsRepository {
26 |
27 | /**
28 | * Saves Analysis results (metrics)
29 | *
30 | * @param resultKey A ResultKey that uniquely identifies a AnalysisResult
31 | * @param analyzerContext The resulting AnalyzerContext of an Analysis
32 | */
33 | def save(resultKey: ResultKey, analyzerContext: AnalyzerContext): Unit
34 |
35 | /**
36 | * Get a AnalyzerContext saved using exactly the same resultKey if present
37 | */
38 | def loadByKey(resultKey: ResultKey): Option[AnalyzerContext]
39 |
40 | /** Get a builder class to construct a loading query to get AnalysisResults */
41 | def load(): MetricsRepositoryMultipleResultsLoader
42 |
43 | }
44 |
45 | /**
46 | * Information that uniquely identifies a AnalysisResult
47 | *
48 | * @param dataSetDate A date related to the AnalysisResult
49 | * @param tags A map with additional annotations
50 | */
51 | case class ResultKey(dataSetDate: Long, tags: Map[String, String] = Map.empty)
52 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/ConstraintSuggestionResult.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions
18 |
19 | import com.amazon.deequ.VerificationResult
20 | import com.amazon.deequ.checks.CheckStatus
21 | import com.amazon.deequ.profiles.{ColumnProfile, ColumnProfiles}
22 |
23 | /**
24 | * The result returned from the ConstraintSuggestionSuite
25 | *
26 | * @param columnProfiles The column profiles
27 | * @param numRecordsUsedForProfiling The number of records that were used for computing
28 | * the column profiles
29 | * @param constraintSuggestions The suggested constraints
30 | * @param verificationResult The verificationResult in case a train/test split was used
31 | */
32 | case class ConstraintSuggestionResult(
33 | columnProfiles: Map[String, ColumnProfile],
34 | numRecordsUsedForProfiling: Long,
35 | constraintSuggestions: Map[String, Seq[ConstraintSuggestion]],
36 | verificationResult: Option[VerificationResult] = None)
37 |
38 |
39 | object ConstraintSuggestionResult {
40 |
41 | def getColumnProfilesAsJson(constraintSuggestionResult: ConstraintSuggestionResult): String = {
42 |
43 | ColumnProfiles
44 | .toJson(constraintSuggestionResult.columnProfiles.values.toSeq)
45 | }
46 |
47 | def getConstraintSuggestionsAsJson(constraintSuggestionResult: ConstraintSuggestionResult)
48 | : String = {
49 | ConstraintSuggestions
50 | .toJson(constraintSuggestionResult.constraintSuggestions.values.fold(Seq.empty)( _ ++ _))
51 | }
52 |
53 | def getEvaluationResultsAsJson(constraintSuggestionResult: ConstraintSuggestionResult)
54 | : String = {
55 |
56 | ConstraintSuggestions
57 | .evaluationResultsToJson(
58 | constraintSuggestionResult.constraintSuggestions.values.fold(Seq.empty)( _ ++ _),
59 | constraintSuggestionResult.verificationResult.getOrElse(
60 | VerificationResult(CheckStatus.Warning, Map.empty, Map.empty)))
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/CompleteIfCompleteRule.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.checks.Check
20 | import com.amazon.deequ.constraints.Constraint.completenessConstraint
21 | import com.amazon.deequ.profiles.ColumnProfile
22 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
23 | import com.amazon.deequ.suggestions.ConstraintSuggestion
24 |
25 | /** If a column is complete in the sample, we suggest a NOT NULL constraint */
26 | case class CompleteIfCompleteRule() extends ConstraintRule[ColumnProfile] {
27 |
28 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
29 | profile.completeness == 1.0
30 | }
31 |
32 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
33 |
34 | val constraint = completenessConstraint(profile.column, Check.IsOne)
35 |
36 | CommonConstraintSuggestion(
37 | constraint,
38 | profile.column,
39 | "Completeness: " + profile.completeness.toString,
40 | s"'${profile.column}' is not null",
41 | this,
42 | s""".isComplete("${profile.column}")"""
43 | )
44 | }
45 |
46 | override val ruleDescription: String = "If a column is complete in the sample, " +
47 | "we suggest a NOT NULL constraint"
48 | }
49 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/ConstraintRule.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.profiles.ColumnProfile
20 | import com.amazon.deequ.suggestions.ConstraintSuggestion
21 |
22 | /** Abstract base class for all constraint suggestion rules */
23 | abstract class ConstraintRule[P <: ColumnProfile] {
24 |
25 | val ruleDescription: String
26 |
27 | /**
28 | * Decide whether the rule should be applied to a particular column
29 | *
30 | * @param profile profile of the column
31 | * @param numRecords overall number of records
32 | * @return
33 | */
34 | def shouldBeApplied(profile: P, numRecords: Long): Boolean
35 |
36 | /**
37 | * Generated a suggested constraint for the column
38 | *
39 | * @param profile profile of the column
40 | * @param numRecords overall number of records
41 | * @return
42 | */
43 | def candidate(profile: P, numRecords: Long): ConstraintSuggestion
44 | }
45 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/HasMax.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.constraints.Constraint.maxConstraint
20 | import com.amazon.deequ.profiles.ColumnProfile
21 | import com.amazon.deequ.profiles.NumericColumnProfile
22 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
23 | import com.amazon.deequ.suggestions.ConstraintSuggestion
24 |
25 | /** If we see only non-negative numbers in a column, we suggest a corresponding
26 | * constraint
27 | */
28 | case class HasMax() extends ConstraintRule[ColumnProfile] {
29 |
30 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
31 | profile match {
32 | case np: NumericColumnProfile => np.maximum.isDefined
33 | case _ => false
34 | }
35 | }
36 |
37 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
38 | val maximum: Double = profile match { case np: NumericColumnProfile => np.maximum.get }
39 |
40 | val description = s"'${profile.column}' <= $maximum"
41 | val constraint = maxConstraint(profile.column, _ == maximum)
42 |
43 | CommonConstraintSuggestion(
44 | constraint,
45 | profile.column,
46 | s"Maximum: $maximum",
47 | description,
48 | this,
49 | s""".hasMax("${profile.column}", _ == $maximum)"""
50 | )
51 | }
52 |
53 | override val ruleDescription: String = "If we see a numeric column, " +
54 | "we suggest a corresponding Maximum value constraint"
55 | }
56 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/HasMaxLength.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2022 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.constraints.Constraint.maxLengthConstraint
20 | import com.amazon.deequ.profiles.ColumnProfile
21 | import com.amazon.deequ.profiles.StringColumnProfile
22 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
23 | import com.amazon.deequ.suggestions.ConstraintSuggestion
24 |
25 | case class HasMaxLength() extends ConstraintRule[ColumnProfile] {
26 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
27 | profile match {
28 | case profile: StringColumnProfile => profile.maxLength.isDefined
29 | case _ => false
30 | }
31 | }
32 |
33 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
34 | val maxLength: Double = profile.asInstanceOf[StringColumnProfile].maxLength.get
35 |
36 | val constraint = maxLengthConstraint(profile.column, _ <= maxLength)
37 |
38 | CommonConstraintSuggestion(
39 | constraint,
40 | profile.column,
41 | "MaxLength: " + profile.completeness.toString,
42 | s"The length of '${profile.column}' <= $maxLength",
43 | this,
44 | s""".hasMaxLength("${profile.column}", _ <= $maxLength)"""
45 | )
46 | }
47 |
48 | override val ruleDescription: String = "If we see a string column, " +
49 | "we suggest a corresponding Maximum length constraint"
50 | }
51 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/HasMean.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.constraints.Constraint.meanConstraint
20 | import com.amazon.deequ.profiles.ColumnProfile
21 | import com.amazon.deequ.profiles.NumericColumnProfile
22 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
23 | import com.amazon.deequ.suggestions.ConstraintSuggestion
24 |
25 | /** If we see only non-negative numbers in a column, we suggest a corresponding
26 | * constraint
27 | */
28 | case class HasMean() extends ConstraintRule[ColumnProfile] {
29 |
30 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
31 | profile match {
32 | case np: NumericColumnProfile => np.mean.isDefined
33 | case _ => false
34 | }
35 | }
36 |
37 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
38 | val mean: Double = profile match { case np: NumericColumnProfile => np.maximum.get }
39 |
40 | val description = s"'${profile.column}' <= $mean"
41 | val constraint = meanConstraint(profile.column, _ == mean)
42 |
43 | CommonConstraintSuggestion(
44 | constraint,
45 | profile.column,
46 | s"Mean: $mean",
47 | description,
48 | this,
49 | s""".hasMean("${profile.column}", _ == $mean)"""
50 | )
51 | }
52 |
53 | override val ruleDescription: String = "If we see a numeric column, " +
54 | "we suggest a corresponding Mean value constraint"
55 | }
56 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/HasMin.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.constraints.Constraint.minConstraint
20 | import com.amazon.deequ.profiles.ColumnProfile
21 | import com.amazon.deequ.profiles.NumericColumnProfile
22 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
23 | import com.amazon.deequ.suggestions.ConstraintSuggestion
24 |
25 | /** If we see only non-negative numbers in a column, we suggest a corresponding
26 | * constraint
27 | */
28 | case class HasMin() extends ConstraintRule[ColumnProfile] {
29 |
30 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
31 | profile match {
32 | case np: NumericColumnProfile => np.minimum.isDefined
33 | case _ => false
34 | }
35 | }
36 |
37 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
38 | val minimum: Double = profile match { case np: NumericColumnProfile => np.minimum.get }
39 |
40 | val description = s"'${profile.column}' >= $minimum"
41 | val constraint = minConstraint(profile.column, _ == minimum)
42 |
43 | CommonConstraintSuggestion(
44 | constraint,
45 | profile.column,
46 | s"Minimum: $minimum",
47 | description,
48 | this,
49 | s""".hasMin("${profile.column}", _ == $minimum)"""
50 | )
51 | }
52 |
53 | override val ruleDescription: String = "If we see a numeric column, " +
54 | "we suggest a corresponding Minimum value constraint"
55 | }
56 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/HasMinLength.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2022 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.constraints.Constraint.minLengthConstraint
20 | import com.amazon.deequ.profiles.ColumnProfile
21 | import com.amazon.deequ.profiles.StringColumnProfile
22 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
23 | import com.amazon.deequ.suggestions.ConstraintSuggestion
24 |
25 | case class HasMinLength() extends ConstraintRule[ColumnProfile] {
26 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
27 | profile match {
28 | case profile: StringColumnProfile => profile.minLength.isDefined
29 | case _ => false
30 | }
31 | }
32 |
33 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
34 | val stringProfile = profile.asInstanceOf[StringColumnProfile]
35 | val minLength: Double = stringProfile.minLength.get
36 |
37 | val constraint = minLengthConstraint(profile.column, _ >= minLength)
38 |
39 | CommonConstraintSuggestion(
40 | constraint,
41 | profile.column,
42 | "MinLength: " + minLength,
43 | s"The length of '${profile.column}' >= $minLength",
44 | this,
45 | s""".hasMinLength("${profile.column}", _ >= $minLength)"""
46 | )
47 | }
48 |
49 | override val ruleDescription: String = "If we see a string column, " +
50 | "we suggest a corresponding Minimum length constraint"
51 | }
52 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/HasStandardDeviation.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.constraints.Constraint.standardDeviationConstraint
20 | import com.amazon.deequ.profiles.ColumnProfile
21 | import com.amazon.deequ.profiles.NumericColumnProfile
22 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
23 | import com.amazon.deequ.suggestions.ConstraintSuggestion
24 |
25 | /** If we see only non-negative numbers in a column, we suggest a corresponding
26 | * constraint
27 | */
28 | case class HasStandardDeviation() extends ConstraintRule[ColumnProfile] {
29 |
30 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
31 | profile match {
32 | case np: NumericColumnProfile => np.mean.isDefined
33 | case _ => false
34 | }
35 | }
36 |
37 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
38 | val stdDev: Double = profile match { case np: NumericColumnProfile => np.stdDev.get }
39 |
40 | val description = s"'${profile.column}' <= $stdDev"
41 | val constraint = standardDeviationConstraint(profile.column, _ == stdDev)
42 |
43 | CommonConstraintSuggestion(
44 | constraint,
45 | profile.column,
46 | s"stdDev: $stdDev",
47 | description,
48 | this,
49 | s""".hasStandardDeviation("${profile.column}", _ == $stdDev)"""
50 | )
51 | }
52 |
53 | override val ruleDescription: String = "If we see a numeric column, " +
54 | "we suggest a corresponding standard deviation value constraint"
55 | }
56 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/NonNegativeNumbersRule.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.checks.Check
20 | import com.amazon.deequ.constraints.Constraint.complianceConstraint
21 | import com.amazon.deequ.profiles.ColumnProfile
22 | import com.amazon.deequ.profiles.NumericColumnProfile
23 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
24 | import com.amazon.deequ.suggestions.ConstraintSuggestion
25 |
26 | /** If we see only non-negative numbers in a column, we suggest a corresponding constraint */
27 | case class NonNegativeNumbersRule() extends ConstraintRule[ColumnProfile] {
28 |
29 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
30 | profile match {
31 | case numericProfile: NumericColumnProfile => numericProfile.minimum.exists(_ >= 0.0)
32 | case _ => false
33 | }
34 | }
35 |
36 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
37 |
38 | val description = s"'${profile.column}' has no negative values"
39 | val constraint = complianceConstraint(description,
40 | s"${profile.column} >= 0",
41 | Check.IsOne,
42 | columns = List(profile.column))
43 |
44 | val minimum = profile match {
45 | case numericProfile: NumericColumnProfile
46 | if numericProfile.minimum.isDefined => numericProfile.minimum.get.toString
47 | case _ => "Error while calculating minimum!"
48 | }
49 |
50 | CommonConstraintSuggestion(
51 | constraint,
52 | profile.column,
53 | "Minimum: " + minimum,
54 | description,
55 | this,
56 | s""".isNonNegative("${profile.column}")"""
57 | )
58 | }
59 |
60 | override val ruleDescription: String = "If we see only non-negative numbers in a " +
61 | "column, we suggest a corresponding constraint"
62 | }
63 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/RetainTypeRule.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.analyzers.DataTypeInstances
20 | import com.amazon.deequ.checks.Check
21 | import com.amazon.deequ.constraints.ConstrainableDataTypes
22 | import com.amazon.deequ.constraints.Constraint.dataTypeConstraint
23 | import com.amazon.deequ.profiles.ColumnProfile
24 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
25 | import com.amazon.deequ.suggestions.ConstraintSuggestion
26 |
27 | /** If we detect a non-string type, we suggest a type constraint */
28 | case class RetainTypeRule() extends ConstraintRule[ColumnProfile] {
29 |
30 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
31 | val testableType = profile.dataType match {
32 | case DataTypeInstances.Integral | DataTypeInstances.Fractional | DataTypeInstances.Boolean =>
33 | true
34 | case _ => false
35 | }
36 |
37 | profile.isDataTypeInferred && testableType
38 | }
39 |
40 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
41 |
42 | val typeToCheck = profile.dataType match {
43 | case DataTypeInstances.Fractional => ConstrainableDataTypes.Fractional
44 | case DataTypeInstances.Integral => ConstrainableDataTypes.Integral
45 | case DataTypeInstances.Boolean => ConstrainableDataTypes.Boolean
46 | }
47 |
48 | val constraint = dataTypeConstraint(profile.column, typeToCheck, Check.IsOne)
49 |
50 | CommonConstraintSuggestion(
51 | constraint,
52 | profile.column,
53 | "DataType: " + profile.dataType.toString,
54 | s"'${profile.column}' has type ${profile.dataType}",
55 | this,
56 | s""".hasDataType("${profile.column}", ConstrainableDataTypes.${profile.dataType})"""
57 | )
58 | }
59 |
60 | override val ruleDescription: String = "If we detect a non-string type, we suggest a " +
61 | "type constraint"
62 | }
63 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/UniqueIfApproximatelyUniqueRule.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules
18 |
19 | import com.amazon.deequ.checks.Check
20 | import com.amazon.deequ.constraints.Constraint.uniquenessConstraint
21 | import com.amazon.deequ.profiles.ColumnProfile
22 | import com.amazon.deequ.suggestions.CommonConstraintSuggestion
23 | import com.amazon.deequ.suggestions.ConstraintSuggestion
24 |
25 | /**
26 | * If the ratio of approximate num distinct values in a column is close to the number of records
27 | * (within error of HLL sketch), we suggest a UNIQUE constraint
28 | */
29 | case class UniqueIfApproximatelyUniqueRule() extends ConstraintRule[ColumnProfile] {
30 |
31 | override def shouldBeApplied(profile: ColumnProfile, numRecords: Long): Boolean = {
32 |
33 | val approximateDistinctness = profile.approximateNumDistinctValues.toDouble / numRecords
34 |
35 | // TODO This bound depends on the error guarantees of the HLL sketch
36 | profile.completeness == 1.0 && math.abs(1.0 - approximateDistinctness) <= 0.08
37 | }
38 |
39 | override def candidate(profile: ColumnProfile, numRecords: Long): ConstraintSuggestion = {
40 |
41 | val constraint = uniquenessConstraint(Seq(profile.column), Check.IsOne)
42 | val approximateDistinctness = profile.approximateNumDistinctValues.toDouble / numRecords
43 |
44 | CommonConstraintSuggestion(
45 | constraint,
46 | profile.column,
47 | "ApproxDistinctness: " + approximateDistinctness.toString,
48 | s"'${profile.column}' is unique",
49 | this,
50 | s""".isUnique("${profile.column}")"""
51 | )
52 | }
53 |
54 | override val ruleDescription: String = "If the ratio of approximate num distinct values " +
55 | "in a column is close to the number of records (within the error of the HLL sketch), " +
56 | "we suggest a UNIQUE constraint"
57 | }
58 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/interval/ConfidenceIntervalStrategy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules.interval
18 |
19 | import breeze.stats.distributions.{Gaussian, Rand}
20 | import com.amazon.deequ.suggestions.rules.interval.ConfidenceIntervalStrategy._
21 |
22 | /**
23 | * Strategy for calculate confidence interval
24 | * */
25 | trait ConfidenceIntervalStrategy {
26 |
27 | /**
28 | * Generated confidence interval interval
29 | * @param pHat sample of the population that share a trait
30 | * @param numRecords overall number of records
31 | * @param confidence confidence level of method used to estimate the interval.
32 | * @return
33 | */
34 | def calculateTargetConfidenceInterval(
35 | pHat: Double,
36 | numRecords: Long,
37 | confidence: Double = defaultConfidence
38 | ): ConfidenceInterval
39 |
40 | def validateInput(pHat: Double, confidence: Double): Unit = {
41 | require(0.0 <= pHat && pHat <= 1.0, "pHat must be between 0.0 and 1.0")
42 | require(0.0 <= confidence && confidence <= 1.0, "confidence must be between 0.0 and 1.0")
43 | }
44 |
45 | def calculateZScore(confidence: Double): Double = Gaussian(0, 1)(Rand).inverseCdf(1 - ((1.0 - confidence)/ 2.0))
46 | }
47 |
48 | object ConfidenceIntervalStrategy {
49 | val defaultConfidence = 0.95
50 | val defaultIntervalStrategy: ConfidenceIntervalStrategy = WaldIntervalStrategy()
51 |
52 | case class ConfidenceInterval(lowerBound: Double, upperBound: Double)
53 | }
54 |
55 |
56 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/interval/WaldIntervalStrategy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules.interval
18 |
19 | import com.amazon.deequ.suggestions.rules.interval.ConfidenceIntervalStrategy.ConfidenceInterval
20 | import com.amazon.deequ.suggestions.rules.interval.ConfidenceIntervalStrategy.defaultConfidence
21 |
22 | import scala.math.BigDecimal.RoundingMode
23 |
24 | /**
25 | * Implements the Wald Interval method for creating a binomial proportion confidence interval. Provided for backwards
26 | * compatibility. using [[WaldIntervalStrategy]] for calculating confidence interval can be problematic when dealing
27 | * with small sample sizes or proportions close to 0 or 1. It also have poorer coverage and might produce confidence
28 | * limit outside the range of [0,1]
29 | * @see
31 | * Normal approximation interval (Wikipedia)
32 | */
33 | @deprecated("WilsonScoreIntervalStrategy is recommended for calculating confidence interval")
34 | case class WaldIntervalStrategy() extends ConfidenceIntervalStrategy {
35 | def calculateTargetConfidenceInterval(
36 | pHat: Double,
37 | numRecords: Long,
38 | confidence: Double = defaultConfidence
39 | ): ConfidenceInterval = {
40 | validateInput(pHat, confidence)
41 | val successRatio = BigDecimal(pHat)
42 | val marginOfError = BigDecimal(calculateZScore(confidence) * math.sqrt(pHat * (1 - pHat) / numRecords))
43 | val lowerBound = (successRatio - marginOfError).setScale(2, RoundingMode.DOWN).toDouble
44 | val upperBound = (successRatio + marginOfError).setScale(2, RoundingMode.UP).toDouble
45 | ConfidenceInterval(lowerBound, upperBound)
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/suggestions/rules/interval/WilsonScoreIntervalStrategy.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules.interval
18 |
19 | import com.amazon.deequ.suggestions.rules.interval.ConfidenceIntervalStrategy.ConfidenceInterval
20 | import com.amazon.deequ.suggestions.rules.interval.ConfidenceIntervalStrategy.defaultConfidence
21 |
22 | import scala.math.BigDecimal.RoundingMode
23 |
24 | /**
25 | * Using Wilson score method for creating a binomial proportion confidence interval.
26 | *
27 | * @see
29 | * Wilson score interval (Wikipedia)
30 | */
31 | case class WilsonScoreIntervalStrategy() extends ConfidenceIntervalStrategy {
32 |
33 | def calculateTargetConfidenceInterval(
34 | pHat: Double, numRecords: Long,
35 | confidence: Double = defaultConfidence
36 | ): ConfidenceInterval = {
37 | validateInput(pHat, confidence)
38 | val zScore = calculateZScore(confidence)
39 | val zSquareOverN = math.pow(zScore, 2) / numRecords
40 | val factor = 1.0 / (1 + zSquareOverN)
41 | val adjustedSuccessRatio = pHat + zSquareOverN/2
42 | val marginOfError = zScore * math.sqrt(pHat * (1 - pHat)/numRecords + zSquareOverN/(4 * numRecords))
43 | val lowerBound = BigDecimal(factor * (adjustedSuccessRatio - marginOfError)).setScale(2, RoundingMode.DOWN).toDouble
44 | val upperBound = BigDecimal(factor * (adjustedSuccessRatio + marginOfError)).setScale(2, RoundingMode.UP).toDouble
45 | ConfidenceInterval(lowerBound, upperBound)
46 | }
47 | }
48 |
--------------------------------------------------------------------------------
/src/main/scala/com/amazon/deequ/utilities/ColumnUtil.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.utilities
18 |
19 | object ColumnUtil {
20 |
21 | def removeEscapeColumn(column: String): String = {
22 | if (column.startsWith("`") && column.endsWith("`")) {
23 | column.substring(1, column.length - 1)
24 | } else {
25 | column
26 | }
27 | }
28 |
29 | def escapeColumn(column: String): String = {
30 | if (column.contains(".")) {
31 | "`" + column + "`"
32 | } else {
33 | column
34 | }
35 | }
36 |
37 | }
38 |
--------------------------------------------------------------------------------
/src/test/resources/EMRSparkShellTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | /*
18 | For testing inside EMR or other flavors of spark cluster. Run commands after building git repo from source.
19 | Add additional test classes as needed
20 | scala 2.12
21 | spark-shell -i /src/test/resources/EMRSparkShellTest.txt \
22 | --packages org.scalatest:scalatest_2.12:3.1.2,org.scalamock:scalamock_2.12:4.4.0,org.scala-lang:scala-compiler:2.12.10,\
23 | org.mockito:mockito-core:2.28.2,org.openjdk.jmh:jmh-core:1.23,org.openjdk.jmh:jmh-generator-annprocess:1.23,org.apache.datasketches:datasketches-java:1.3.0-incubating \
24 | --jars /target/deequ_2.12-1.1.0-SNAPSHOT.jar,/target/deequ_2.12-1.1.0-SNAPSHOT-tests.jar
25 |
26 | scala 2.11
27 | spark-shell -i /src/test/resources/EMRSparkShellTest.txt \
28 | --packages org.scalatest:scalatest_2.11:3.1.2,org.scalamock:scalamock_2.11:4.4.0,org.scala-lang:scala-compiler:2.11.10,\
29 | org.mockito:mockito-core:2.28.2,org.openjdk.jmh:jmh-core:1.23,org.openjdk.jmh:jmh-generator-annprocess:1.23,org.apache.datasketches:datasketches-java:1.3.0-incubating \
30 | --jars /target/deequ-1.1.0-SNAPSHOT.jar,/target/spark-deequ-testing/deequ-1.1.0-SNAPSHOT-tests.jar
31 | */
32 |
33 | import com.amazon.deequ.analyzers.{AnalysisTest, AnalyzerTests, IncrementalAnalysisTest}
34 | import com.amazon.deequ.analyzers.runners.{AnalysisRunnerTests, AnalyzerContextTest}
35 | import com.amazon.deequ.{VerificationResultTest, VerificationSuiteTest}
36 |
37 | (new VerificationSuiteTest).execute()
38 | (new VerificationResultTest).execute()
39 | (new AnalysisRunnerTests).execute()
40 | (new AnalyzerContextTest).execute()
41 | (new AnalysisTest).execute()
42 | (new AnalyzerTests).execute()
43 | (new IncrementalAnalysisTest).execute()
44 | //Add additional test classes as needed
45 |
--------------------------------------------------------------------------------
/src/test/resources/log4j.properties:
--------------------------------------------------------------------------------
1 | # Change this to set Spark log level
2 | log4j.logger.org.apache.spark=WARN
3 |
4 | # Silence akka remoting
5 | log4j.logger.Remoting=WARN
6 |
7 | # Ignore messages below warning level from Jetty, because it's a bit verbose
8 | log4j.logger.org.eclipse.jetty=WARN
9 |
10 | # INFO log level not required for tests
11 | log4j.logger.org.apache=WARN
12 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/DatatypeSuggestionTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ
18 |
19 | import com.amazon.deequ.profiles.StringColumnProfile
20 | import com.amazon.deequ.profiles.ColumnProfiler
21 | import com.amazon.deequ.utils.FixtureSupport
22 | import org.scalamock.scalatest.MockFactory
23 | import org.scalatest.{Matchers, WordSpec}
24 |
25 | class DatatypeSuggestionTest extends WordSpec with Matchers with SparkContextSpec
26 | with FixtureSupport with MockFactory{
27 |
28 | "Column Profiler" should {
29 | "return the correct datatype(String) in case of profiling empty string columns" in
30 | withSparkSession { sparkSession =>
31 |
32 | val df = getEmptyColumnDataDf(sparkSession = sparkSession)
33 |
34 | val profile = ColumnProfiler
35 | .profile(df, Option(Seq("att1")))
36 | .profiles("att1")
37 |
38 | assert(profile.isInstanceOf[StringColumnProfile])
39 | assert(profile.isDataTypeInferred && profile.dataType.toString.equalsIgnoreCase("String"))
40 | }
41 | }
42 | }
43 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/KLL/KLLBenchmark.java:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.KLL;
18 |
19 | import com.amazon.deequ.analyzers.QuantileNonSample;
20 | import org.apache.datasketches.kll.KllFloatsSketch;
21 | import org.openjdk.jmh.annotations.Benchmark;
22 | import org.openjdk.jmh.annotations.BenchmarkMode;
23 | import org.openjdk.jmh.annotations.Fork;
24 | import org.openjdk.jmh.annotations.Mode;
25 | import org.openjdk.jmh.annotations.OutputTimeUnit;
26 | import org.openjdk.jmh.infra.Blackhole;
27 | import org.openjdk.jmh.runner.Runner;
28 | import org.openjdk.jmh.runner.RunnerException;
29 | import org.openjdk.jmh.runner.options.Options;
30 | import org.openjdk.jmh.runner.options.OptionsBuilder;
31 |
32 | import java.util.Random;
33 | import java.util.concurrent.TimeUnit;
34 |
35 | @BenchmarkMode(Mode.AverageTime)
36 | @OutputTimeUnit(TimeUnit.MILLISECONDS)
37 | @Fork(value = 2, jvmArgs = {"-Xms2G", "-Xmx2G"})
38 | public class KLLBenchmark {
39 |
40 | private static final int N = 10_000_000;
41 |
42 | private static float[] DATA_FOR_TESTING = createData();
43 |
44 | public static void main(String[] args) throws RunnerException {
45 |
46 | Options opt = new OptionsBuilder()
47 | .include(KLLBenchmark.class.getSimpleName())
48 | .forks(1)
49 | .build();
50 |
51 | new Runner(opt).run();
52 | }
53 |
54 | private static float[] createData() {
55 | Random prng = new Random();
56 | float[] numbers = new float[N];
57 | for (int i = 0; i < N; i++) {
58 | numbers[i] = prng.nextFloat();
59 | }
60 | return numbers;
61 | }
62 |
63 | @Benchmark
64 | public void sumArray(Blackhole bh) {
65 | float sum = 0.0f;
66 | for (int i = 0; i < N; i++) {
67 | sum += DATA_FOR_TESTING[i];
68 | }
69 | bh.consume(sum);
70 | }
71 |
72 | @Benchmark
73 | public void sketchArrayWithKLL(Blackhole bh) {
74 | QuantileNonSample sketch = KLLBenchmarkHelper.floatSketch();
75 | for (int i = 0; i < N; i++) {
76 | sketch.update(DATA_FOR_TESTING[i]);
77 | }
78 | bh.consume(sketch);
79 | }
80 |
81 | @Benchmark
82 | public void sketchArrayWithJavaSketchesKLL(Blackhole bh) {
83 | KllFloatsSketch sketch = new KllFloatsSketch();
84 | for (int i = 0; i < N; i++) {
85 | sketch.update(DATA_FOR_TESTING[i]);
86 | }
87 | bh.consume(sketch);
88 | }
89 | }
90 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/KLL/KLLBenchmarkHelper.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.KLL
18 |
19 | import com.amazon.deequ.analyzers.{KLLSketch, QuantileNonSample}
20 |
21 | object KLLBenchmarkHelper {
22 |
23 | def floatSketch(): QuantileNonSample[java.lang.Float] = {
24 | new QuantileNonSample[java.lang.Float](KLLSketch.DEFAULT_SKETCH_SIZE,
25 | KLLSketch.DEFAULT_SHRINKING_FACTOR)
26 | }
27 |
28 | }
29 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/SparkBasicTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ
18 |
19 | import org.scalatest.{Matchers, WordSpec}
20 |
21 | class SparkBasicTest extends WordSpec with Matchers with SparkContextSpec {
22 | "check that initializing a spark context and a basic example works" in
23 | withSparkSession { sparkSession =>
24 | val sc = sparkSession.sparkContext
25 | val xs = sc.parallelize(1 to 100)
26 | val res = xs.sum()
27 | res should be(5050)
28 | }
29 |
30 | "check that monitoring spark session works" in
31 | withMonitorableSparkSession { (sparkSession, sparkMonitor) =>
32 | val sc = sparkSession.sparkContext
33 | val xs = sc.parallelize(1 to 100)
34 |
35 |
36 | (1 to 2).foreach { index =>
37 | val res = sparkMonitor.withMonitoringSession { stat =>
38 | val sum = xs.map(_ * index).sum()
39 | // Spark jobs are running in different monitoring sessions
40 | assert(stat.jobCount == 1)
41 | sum
42 | }
43 | res should be(5050 * index)
44 | }
45 |
46 | sparkMonitor.withMonitoringSession { stat =>
47 | (1 to 2).foreach { index =>
48 | xs.map(_ * index).sum()
49 | }
50 | // Spark jobs are running in the same monitoring session
51 | assert(stat.jobCount == 2)
52 | }
53 | }
54 | }
55 |
56 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/SparkMonitor.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ
18 |
19 | import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerStageCompleted, StageInfo}
20 |
21 | /**
22 | * A class representing a statistics about a sparkSession.
23 | * Currently, only number of spark jobs submitted and its stages are being tracked.
24 | */
25 | class SparkSessionStats {
26 | private var numberOfJobsSubmitted = 0
27 | private var stageInfos = Seq[StageInfo]()
28 |
29 | def jobCount: Int = {
30 | numberOfJobsSubmitted
31 | }
32 |
33 | def allExecutedStages: Seq[StageInfo] = {
34 | stageInfos
35 | }
36 |
37 | def recordJobStart(jobStart: SparkListenerJobStart): Unit = {
38 | numberOfJobsSubmitted += 1
39 | }
40 |
41 | def recordStageInfos(stageInfo: StageInfo): Unit = {
42 | stageInfos = stageInfos :+ stageInfo
43 | }
44 |
45 | def reset(): Unit = {
46 | numberOfJobsSubmitted = 0
47 | stageInfos = Seq[StageInfo]()
48 | }
49 |
50 | }
51 |
52 | /**
53 | * A SparkListener implementation to monitor spark jobs submitted
54 | */
55 | class SparkMonitor extends SparkListener {
56 | val stat = new SparkSessionStats
57 |
58 | override def onJobStart(jobStart: SparkListenerJobStart) {
59 | stat.recordJobStart(jobStart)
60 | println(s"Job started with ${jobStart.stageInfos.size} stages: $jobStart " +
61 | s"details : ${jobStart.stageInfos.map(_.name)}")
62 |
63 | }
64 |
65 | override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = {
66 | stat.recordStageInfos(stageCompleted.stageInfo)
67 | println(s"Stage ${stageCompleted.stageInfo.stageId} completed with " +
68 | s"${stageCompleted.stageInfo.numTasks} tasks.")
69 | }
70 |
71 | /**
72 | * @param testFun thunk to run with SparkSessionStats as an argument.
73 | * Provides a monitoring session where the stats are being reset at the beginning
74 | *
75 | */
76 | def withMonitoringSession(testFun: (SparkSessionStats) => Any): Any = {
77 | stat.reset
78 | testFun(stat)
79 | }
80 |
81 | }
82 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/SuggestionAndVerificationIntegrationTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ
18 |
19 | import com.amazon.deequ.checks.Check
20 | import com.amazon.deequ.checks.CheckLevel
21 | import com.amazon.deequ.checks.CheckStatus
22 | import com.amazon.deequ.constraints.Constraint
23 | import com.amazon.deequ.suggestions.ConstraintSuggestionRunner
24 | import com.amazon.deequ.suggestions.Rules
25 | import com.amazon.deequ.suggestions.rules.UniqueIfApproximatelyUniqueRule
26 | import com.amazon.deequ.utilities.ColumnUtil.escapeColumn
27 | import com.amazon.deequ.utils.FixtureSupport
28 | import org.apache.spark.sql.DataFrame
29 | import org.scalatest.Matchers
30 | import org.scalatest.WordSpec
31 |
32 | class SuggestionAndVerificationIntegrationTest extends WordSpec with Matchers with SparkContextSpec
33 | with FixtureSupport {
34 |
35 | "SuggestionAndVerificationIntegration" should {
36 | "Succeed for all constraints suggested for the data with . in column name" in
37 | withSparkSession { session =>
38 |
39 | def assertStatusFor(data: DataFrame, checks: Check*)
40 | (expectedStatus: CheckStatus.Value)
41 | : Unit = {
42 | val verificationSuiteStatus =
43 | VerificationSuite().onData(data).addChecks(checks).run().status
44 | assert(verificationSuiteStatus == expectedStatus)
45 | }
46 |
47 | val data = getDfWithPeriodInName(session)
48 |
49 | val results = ConstraintSuggestionRunner()
50 | .onData(data)
51 | .addConstraintRules(Rules.DEFAULT)
52 | .addConstraintRule(UniqueIfApproximatelyUniqueRule())
53 | .run()
54 |
55 | val columns = data.columns.map { c =>
56 | escapeColumn(c)
57 | }
58 |
59 | val constraints: Seq[Constraint] = columns.flatMap { column =>
60 | results.constraintSuggestions
61 | .getOrElse(column, Seq())
62 | .map(suggestion => suggestion.constraint)
63 | }
64 |
65 | val checksToSucceed = Check(CheckLevel.Error, "group-1", constraints)
66 |
67 | assertStatusFor(data, checksToSucceed)(CheckStatus.Success)
68 | }
69 | }
70 | }
71 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/analyzers/ColumnCountTest.scala:
--------------------------------------------------------------------------------
1 | /*
2 | * Copyright 2024 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
12 | * either express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | *
16 | */
17 |
18 | package com.amazon.deequ.analyzers
19 |
20 | import com.amazon.deequ.SparkContextSpec
21 | import com.amazon.deequ.utils.FixtureSupport
22 | import org.apache.spark.sql.Row
23 | import org.apache.spark.sql.types.StructType
24 | import org.scalatest.matchers.should.Matchers
25 | import org.scalatest.wordspec.AnyWordSpec
26 |
27 | import scala.util.Failure
28 | import scala.util.Success
29 |
30 | class ColumnCountTest extends AnyWordSpec with Matchers with SparkContextSpec with FixtureSupport {
31 | "ColumnCount" should {
32 | "return column count for a dataset" in withSparkSession { session =>
33 | val data = getDfWithStringColumns(session)
34 | val colCount = ColumnCount()
35 |
36 | val state = colCount.computeStateFrom(data)
37 | state.isDefined shouldBe true
38 | state.get.metricValue() shouldBe 5.0
39 |
40 | val metric = colCount.computeMetricFrom(state)
41 | metric.fullColumn shouldBe None
42 | metric.value shouldBe Success(5.0)
43 | }
44 | }
45 | }
46 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/analyzers/MaximumTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.SparkContextSpec
20 | import com.amazon.deequ.metrics.DoubleMetric
21 | import com.amazon.deequ.metrics.FullColumn
22 | import com.amazon.deequ.utils.FixtureSupport
23 | import org.apache.spark.sql.Column
24 | import org.apache.spark.sql.DataFrame
25 | import org.apache.spark.sql.Row
26 | import org.apache.spark.sql.functions.element_at
27 | import org.apache.spark.sql.types.DoubleType
28 | import org.scalatest.matchers.should.Matchers
29 | import org.scalatest.wordspec.AnyWordSpec
30 |
31 | class MaximumTest extends AnyWordSpec with Matchers with SparkContextSpec with FixtureSupport {
32 | private val tempColName = "new"
33 |
34 | private def getValuesDF(df: DataFrame, outcomeColumn: Column): Seq[Row] = {
35 | df.withColumn(tempColName, element_at(outcomeColumn, 2).cast(DoubleType)).collect()
36 | }
37 |
38 | "Max" should {
39 | "return row-level results for columns" in withSparkSession { session =>
40 |
41 | val data = getDfWithNumericValues(session)
42 |
43 | val att1Maximum = Maximum("att1")
44 | val state: Option[MaxState] = att1Maximum.computeStateFrom(data)
45 | val metric: DoubleMetric with FullColumn = att1Maximum.computeMetricFrom(state)
46 |
47 | val values = getValuesDF(data, metric.fullColumn.get).map(_.getAs[Double](tempColName))
48 | values shouldBe Seq(1.0, 2.0, 3.0, 4.0, 5.0, 6.0)
49 | }
50 |
51 | "return row-level results for columns with null" in withSparkSession { session =>
52 | val data = getDfWithNumericValues(session)
53 |
54 | val att1Maximum = Maximum("attNull")
55 | val state: Option[MaxState] = att1Maximum.computeStateFrom(data)
56 | val metric: DoubleMetric with FullColumn = att1Maximum.computeMetricFrom(state)
57 |
58 | val values = getValuesDF(data, metric.fullColumn.get)
59 | .map(r => if (r == null) null else r.getAs[Double](tempColName))
60 | values shouldBe Seq(null, null, null, 5.0, 6.0, 7.0)
61 | }
62 | }
63 | }
64 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/analyzers/MinimumTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2023 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.SparkContextSpec
20 | import com.amazon.deequ.metrics.DoubleMetric
21 | import com.amazon.deequ.metrics.FullColumn
22 | import com.amazon.deequ.utils.FixtureSupport
23 | import org.apache.spark.sql.Column
24 | import org.apache.spark.sql.DataFrame
25 | import org.apache.spark.sql.Row
26 | import org.apache.spark.sql.functions.element_at
27 | import org.apache.spark.sql.types.DoubleType
28 | import org.scalatest.matchers.should.Matchers
29 | import org.scalatest.wordspec.AnyWordSpec
30 |
31 | class MinimumTest extends AnyWordSpec with Matchers with SparkContextSpec with FixtureSupport {
32 | private val tempColName = "new"
33 |
34 | private def getValuesDF(df: DataFrame, outcomeColumn: Column): Seq[Row] = {
35 | df.withColumn(tempColName, element_at(outcomeColumn, 2).cast(DoubleType)).collect()
36 | }
37 |
38 | "Min" should {
39 | "return row-level results for columns" in withSparkSession { session =>
40 | val data = getDfWithNumericValues(session)
41 |
42 | val att1Minimum = Minimum("att1")
43 | val state: Option[MinState] = att1Minimum.computeStateFrom(data)
44 | val metric: DoubleMetric with FullColumn = att1Minimum.computeMetricFrom(state)
45 |
46 | val values = getValuesDF(data, metric.fullColumn.get).map(_.getAs[Double](tempColName))
47 | values shouldBe Seq(1.0, 2.0, 3.0, 4.0, 5.0, 6.0)
48 | }
49 |
50 | "return row-level results for columns with null" in withSparkSession { session =>
51 | val data = getDfWithNumericValues(session)
52 |
53 | val att1Minimum = Minimum("attNull")
54 | val state: Option[MinState] = att1Minimum.computeStateFrom(data)
55 | val metric: DoubleMetric with FullColumn = att1Minimum.computeMetricFrom(state)
56 |
57 | val values = getValuesDF(data, metric.fullColumn.get)
58 | .map(r => if (r == null) null else r.getAs[Double](tempColName))
59 | values shouldBe Seq(null, null, null, 5.0, 6.0, 7.0)
60 | }
61 | }
62 | }
63 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/analyzers/StatesTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.analyzers
18 |
19 | import com.amazon.deequ.SparkContextSpec
20 | import com.amazon.deequ.utils.FixtureSupport
21 | import org.scalatest.matchers.should.Matchers
22 | import org.scalatest.wordspec.AnyWordSpec
23 |
24 | class StatesTest extends AnyWordSpec with Matchers with SparkContextSpec with FixtureSupport {
25 |
26 | "FrequenciesAndNumRows" should {
27 | "merge correctly" in withSparkSession { session =>
28 |
29 | import session.implicits._
30 |
31 | val dataA = Seq("A", "A", "B").toDF("att1")
32 | val dataB = Seq("A", "C", "C").toDF("att1")
33 |
34 | val stateA = FrequencyBasedAnalyzer.computeFrequencies(dataA, "att1" :: Nil)
35 | val stateB = FrequencyBasedAnalyzer.computeFrequencies(dataB, "att1" :: Nil)
36 |
37 | val stateAB = stateA.sum(stateB)
38 |
39 | println(stateA.frequencies.schema)
40 | stateA.frequencies.collect().foreach { println }
41 | println()
42 |
43 | println(stateB.frequencies.schema)
44 | stateB.frequencies.collect().foreach { println }
45 | println()
46 |
47 | println(stateAB.frequencies.schema)
48 | stateAB.frequencies.collect().foreach { println }
49 |
50 | val mergedFrequencies = stateAB.frequencies.collect()
51 | .map { row => row.getString(0) -> row.getLong(1) }
52 | .toMap
53 |
54 | assert(mergedFrequencies.size == 3)
55 | assert(mergedFrequencies.get("A").contains(3))
56 | assert(mergedFrequencies.get("B").contains(1))
57 | assert(mergedFrequencies.get("C").contains(2))
58 | }
59 | }
60 | }
61 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/anomalydetection/AnomalyDetectionTestUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | import scala.util.matching.Regex
20 |
21 | /**
22 | * Utilities to test Anomaly Detection methods and related modules
23 | */
24 | object AnomalyDetectionTestUtils {
25 |
26 | private val numericalValueRegex: Regex = """([+-]?([0-9]*[.])?[0-9]+([Ee][0-9]+)?)""".r
27 |
28 | /**
29 | * Finds the first numerical value in a string
30 | *
31 | * @param details The string containing a numerical value
32 | * @throws IllegalArgumentException Thrown if no value could be found
33 | * @return The value itself
34 | */
35 | def firstDoubleFromString(details: String): Double = {
36 | val firstValue = numericalValueRegex.findFirstIn(details)
37 |
38 | require(firstValue.isDefined, "Input string did not contain a numerical value")
39 |
40 | firstValue.get.toString.toDouble
41 | }
42 |
43 | /**
44 | * Finds the first three numerical values in a string
45 | *
46 | * @param details The string containing at least three numerical values
47 | * @throws IllegalArgumentException Thrown if less than 3 values could be found
48 | * @return The values themselves
49 | */
50 | def firstThreeDoublesFromString(details: String): (Double, Double, Double) = {
51 | val values = numericalValueRegex.findAllIn(details).toVector.map(_.toString.toDouble)
52 |
53 | require(values.length >= 3, "Input string did not contain at least 3 numerical values.")
54 |
55 | (values(0), values(1), values(2))
56 | }
57 | }
58 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/anomalydetection/AnomalyDetectionTestUtilsTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | import org.scalatest.{Matchers, WordSpec}
20 |
21 | class AnomalyDetectionTestUtilsTest extends WordSpec with Matchers {
22 |
23 | "AnomalyDetectionTestUtilsTest" should {
24 |
25 | "throw an exception if no value found" in {
26 | intercept[IllegalArgumentException] {
27 | AnomalyDetectionTestUtils.firstDoubleFromString("noNumber")
28 | }
29 | intercept[IllegalArgumentException] {
30 | AnomalyDetectionTestUtils.firstThreeDoublesFromString("noNumber")
31 | }
32 | }
33 |
34 | "find first value" in {
35 | val str = "xx3.141yyu4.2"
36 | val value = AnomalyDetectionTestUtils.firstDoubleFromString(str)
37 | assert(value == 3.141)
38 | }
39 |
40 | "find all 3 values" in {
41 | val str = "In this 1 string are 3.000 values, not 42.01"
42 |
43 | val (first, second, third) = AnomalyDetectionTestUtils.firstThreeDoublesFromString(str)
44 | assert(first === 1)
45 | assert(second === 3.0)
46 | assert(third === 42.01)
47 | }
48 | }
49 | }
50 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/anomalydetection/HistoryUtilsTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | import com.amazon.deequ.metrics.{DoubleMetric, Entity}
20 | import org.scalatest.{Matchers, WordSpec}
21 |
22 | import scala.util.{Failure, Success}
23 |
24 | class HistoryUtilsTest extends WordSpec with Matchers {
25 |
26 | "History Utils" should {
27 | val sampleException = new IllegalArgumentException()
28 |
29 | val noneMetric = None
30 | val metricWithNoValue = Some(DoubleMetric(Entity.Column, "metric-name", "instance-name",
31 | Failure(sampleException)))
32 | val metricWithValue = Some(DoubleMetric(Entity.Column, "metric-name", "instance-name",
33 | Success(50)))
34 |
35 | "extract optinal metric value" in {
36 | assert(HistoryUtils.extractMetricValue[Double](noneMetric).isEmpty)
37 | assert(HistoryUtils.extractMetricValue[Double](metricWithNoValue).isEmpty)
38 | assert(HistoryUtils.extractMetricValue[Double](metricWithValue).contains(50))
39 |
40 | }
41 | "extract optinal metric values" in {
42 | val metrics = Seq(0L -> noneMetric, 1L -> metricWithNoValue, 2L -> metricWithValue)
43 | assert(HistoryUtils.extractMetricValues[Double](metrics) == Seq(DataPoint[Double](0L, None),
44 | DataPoint[Double](1L, None), DataPoint[Double](2, Some(50))))
45 | }
46 | }
47 | }
48 |
49 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/anomalydetection/RateOfChangeStrategyTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | import org.scalatest.{Matchers, WordSpec}
20 |
21 | /**
22 | * The tested class RateOfChangeStrategy is deprecated.
23 | * This test is to ensure backwards compatibility for deequ checks that still rely on this strategy.
24 | */
25 | class RateOfChangeStrategyTest extends WordSpec with Matchers {
26 |
27 | "RateOfChange Strategy" should {
28 |
29 | val strategy = RateOfChangeStrategy(Some(-2.0), Some(2.0))
30 | val data = (for (i <- 0 to 50) yield {
31 | if (i < 20 || i > 30) {
32 | 1.0
33 | } else {
34 | if (i % 2 == 0) i else -i
35 | }
36 | }).toVector
37 |
38 | "detect all anomalies if no interval specified" in {
39 | val anomalyResult = strategy.detect(data)
40 | val expected = for (i <- 20 to 31) yield {
41 | (i, Anomaly(Option(data(i)), 1.0))
42 | }
43 | assert(anomalyResult == expected)
44 | }
45 | }
46 | }
47 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/anomalydetection/SimpleThresholdStrategyTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.anomalydetection
18 |
19 | import org.scalatest.{Matchers, WordSpec}
20 |
21 | class SimpleThresholdStrategyTest extends WordSpec with Matchers {
22 |
23 | "Simple Threshold Strategy" should {
24 |
25 | val strategy = SimpleThresholdStrategy(upperBound = 1.0)
26 | val data = Vector(-1.0, 2.0, 3.0, 0.5)
27 | val expected = Seq((1, Anomaly(Option(2.0), 1.0)), (2, Anomaly(Option(3.0), 1.0)))
28 |
29 | "detect values above threshold" in {
30 | val anomalyResult = strategy.detect(data, (0, 4))
31 |
32 | assert(anomalyResult == expected)
33 | }
34 |
35 | "detect all values without range specified" in {
36 | val anomalyResult = strategy.detect(data)
37 |
38 | assert(anomalyResult == expected)
39 | }
40 |
41 | "work fine with empty input" in {
42 | val emptySeries = Vector[Double]()
43 | val anomalyResult = strategy.detect(emptySeries)
44 |
45 | assert(anomalyResult == Seq[(Int, Anomaly)]())
46 | }
47 |
48 | "work with upper and lower threshold" in {
49 | val tS = SimpleThresholdStrategy(lowerBound = -0.5, upperBound = 1.0)
50 | val anomalyResult = tS.detect(data)
51 |
52 | assert(anomalyResult == Seq((0, Anomaly(Option(-1.0), 1.0)),
53 | (1, Anomaly(Option(2.0), 1.0)), (2, Anomaly(Option(3.0), 1.0))))
54 | }
55 |
56 | "throw an error when thresholds are not ordered " in {
57 | intercept[IllegalArgumentException] {
58 | val ts = SimpleThresholdStrategy(lowerBound = 2.0, upperBound = 1.0)
59 | }
60 | }
61 |
62 | "produce error message with correct value and bounds" in {
63 | val result = strategy.detect(data)
64 |
65 | result.foreach { case (_, anom) =>
66 | val (value, lowerBound, upperBound) =
67 | AnomalyDetectionTestUtils.firstThreeDoublesFromString(anom.detail.get)
68 |
69 | assert(anom.value.isDefined && value === anom.value.get)
70 | assert(value < lowerBound || value > upperBound)
71 | }
72 | }
73 | }
74 | }
75 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/checks/ColumnConditionTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.checks
18 |
19 |
20 | import org.scalatest.wordspec.AnyWordSpec
21 |
22 | class ColumnConditionTest extends AnyWordSpec {
23 |
24 | "ColumnCondition" should {
25 |
26 | "return the correct isEachNotNull condition" in {
27 | assert(
28 | ColumnCondition.isEachNotNull(Seq("att1", "att2", "att3")) ==
29 | "(((att1 IS NOT NULL) AND (att2 IS NOT NULL)) AND (att3 IS NOT NULL))"
30 | )
31 | }
32 |
33 | "return the correct isAnyNotNull condition" in {
34 | assert(
35 | ColumnCondition.isAnyNotNull(Seq("att1", "att2", "att3")) ==
36 | "(((att1 IS NOT NULL) OR (att2 IS NOT NULL)) OR (att3 IS NOT NULL))"
37 | )
38 | }
39 | }
40 |
41 | }
42 |
43 |
44 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/checks/FilterableCheckTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ
18 | package checks
19 |
20 | import com.amazon.deequ.analyzers.{Completeness, Compliance}
21 | import com.amazon.deequ.utils.FixtureSupport
22 | import org.scalatest.matchers.should.Matchers
23 | import org.scalatest.wordspec.AnyWordSpec
24 |
25 |
26 | class FilterableCheckTest extends AnyWordSpec
27 | with Matchers
28 | with SparkContextSpec
29 | with FixtureSupport {
30 |
31 | "Filterable checks" should {
32 | "build correctly" in {
33 |
34 | val check = Check(CheckLevel.Error, "someCheck")
35 | .isComplete("col1")
36 | .isComplete("col2").where("marketplace = 'EU'")
37 | .hasCompleteness("col3", _ >= 0.9).where("marketplace = 'NA'")
38 | .satisfies("someCol > 5", "const1", columns = List("someCol"))
39 | .satisfies("someCol > 10", "const2", columns = List("someCol")).where("marketplace = 'EU'")
40 |
41 | val completenessAnalyzers =
42 | check.requiredAnalyzers()
43 | .filter { _.isInstanceOf[Completeness] }
44 | .map { _.asInstanceOf[Completeness] }
45 | .toArray
46 | .sortBy { _.column }
47 |
48 | assert(completenessAnalyzers.length == 3)
49 | assert(completenessAnalyzers.head.where.isEmpty)
50 | assert(completenessAnalyzers(1).where.contains("marketplace = 'EU'"))
51 | assert(completenessAnalyzers(2).where.contains("marketplace = 'NA'"))
52 |
53 | val complianceAnalyzers =
54 | check.requiredAnalyzers()
55 | .filter { _.isInstanceOf[Compliance] }
56 | .map { _.asInstanceOf[Compliance] }
57 | .toArray
58 | .sortBy { _.instance }
59 |
60 | assert(complianceAnalyzers.length == 2)
61 | assert(complianceAnalyzers.head.where.isEmpty)
62 | assert(complianceAnalyzers(1).where.contains("marketplace = 'EU'"))
63 | }
64 | }
65 |
66 | }
67 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/constraints/ConstraintUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.constraints
18 |
19 | import org.apache.spark.sql.DataFrame
20 |
21 | object ConstraintUtils {
22 |
23 | def calculate(constraint: Constraint, df: DataFrame): ConstraintResult = {
24 |
25 | val analysisBasedConstraint = constraint match {
26 | case nc: ConstraintDecorator => nc.inner
27 | case c: Constraint => c
28 | }
29 |
30 | analysisBasedConstraint.asInstanceOf[AnalysisBasedConstraint[_, _, _]].calculateAndEvaluate(df)
31 | }
32 | }
33 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/dqdl/DefaultDQDLParserTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl
18 |
19 | import com.amazon.deequ.dqdl.util.DefaultDQDLParser
20 | import org.scalatest.matchers.should.Matchers
21 | import org.scalatest.wordspec.AnyWordSpec
22 | import software.amazon.glue.dqdl.parser.DQDLParser
23 |
24 | import scala.jdk.CollectionConverters.collectionAsScalaIterableConverter
25 |
26 |
27 | class DefaultDQDLParserTest extends AnyWordSpec with Matchers {
28 |
29 | val parser = new DQDLParser()
30 |
31 | "DQDL Parser" should {
32 | "parse valid DQDL rules" in {
33 |
34 | val ruleset = "Rules = [ RowCount > 1, ColumnCount = 3]"
35 | val dqRuleset = parser.parse(ruleset)
36 | val rules = dqRuleset.getRules.asScala
37 |
38 | // Test number of rules
39 | rules.size shouldBe 2
40 |
41 | // Test individual rules
42 | val rowCountRule = rules.find(_.getRuleType == "RowCount")
43 | rowCountRule.isDefined shouldBe true
44 | rowCountRule.map(_.toString) shouldBe Some("RowCount > 1")
45 |
46 | val columnCountRule = rules.find(_.getRuleType == "ColumnCount")
47 | columnCountRule.isDefined shouldBe true
48 | columnCountRule.map(_.toString) shouldBe Some("ColumnCount = 3")
49 | }
50 |
51 | "throw an IllegalArgumentException when DQDL can not be parsed" in {
52 | val thrown = intercept[IllegalArgumentException] {
53 | DefaultDQDLParser.parse("invalid")
54 | }
55 | thrown.getMessage should include("Parsing Error")
56 | }
57 |
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/dqdl/translation/DQDLRuleTranslatorSpec.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.dqdl.translation
18 |
19 | import com.amazon.deequ.dqdl.model.DeequExecutableRule
20 | import com.amazon.deequ.utils.ConditionUtils.ConditionAsString
21 | import org.scalatest.matchers.should.Matchers
22 | import org.scalatest.wordspec.AnyWordSpec
23 | import software.amazon.glue.dqdl.model.DQRule
24 | import com.amazon.deequ.dqdl.model.UnsupportedExecutableRule
25 | import com.amazon.deequ.dqdl.util.DefaultDQDLParser
26 | import software.amazon.glue.dqdl.model.DQRuleset
27 |
28 |
29 | import scala.jdk.CollectionConverters.mapAsJavaMapConverter
30 |
31 |
32 | class DQDLRuleTranslatorSpec extends AnyWordSpec with Matchers {
33 |
34 | "DQDL rules translator" should {
35 | "translate RowCount rule" in {
36 | // given
37 | val parameters: Map[String, String] = Map.empty
38 | val rule: DQRule = new DQRule("RowCount", parameters.asJava, ">100".asCondition)
39 |
40 | // when
41 | val deequRuleOpt: Option[DeequExecutableRule] = DQDLRuleTranslator.translateRule(rule).toOption
42 |
43 | // then
44 | deequRuleOpt shouldBe defined
45 | deequRuleOpt.get.check.toString should include("SizeConstraint")
46 | }
47 | }
48 |
49 | "get executable rules for RowCount" in {
50 | // given
51 | val ruleset: DQRuleset = DefaultDQDLParser.parse("Rules=[RowCount > 10]")
52 |
53 | // when
54 | val rules = DQDLRuleTranslator.toExecutableRules(ruleset)
55 |
56 | // then
57 | rules.size should equal(1)
58 | val rule = rules.head
59 | rule.evaluatedMetricName.get should equal("Dataset.*.RowCount")
60 | rule.dqRule.getRuleType should equal("RowCount")
61 | }
62 |
63 | "get unknown executable rule" in {
64 | // given
65 | val ruleset: DQRuleset = DefaultDQDLParser.parse("Rules=[Completeness \"Name\" > 0.8]")
66 |
67 | // when
68 | val rules = DQDLRuleTranslator.toExecutableRules(ruleset)
69 |
70 | // then
71 | rules.size should equal(1)
72 | val rule = rules.head
73 | rule shouldBe an[UnsupportedExecutableRule]
74 | rule.evaluatedMetricName should equal(None)
75 | }
76 | }
77 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/examples/ExamplesTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.examples
18 |
19 | import org.scalatest.WordSpec
20 |
21 | class ExamplesTest extends WordSpec {
22 |
23 | "all examples" should {
24 | "run without errors" in {
25 | BasicExample.main(Array.empty)
26 | IncrementalMetricsExample.main(Array.empty)
27 | MetricsRepositoryExample.main(Array.empty)
28 | UpdateMetricsOnPartitionedDataExample.main(Array.empty)
29 | DataProfilingExample.main(Array.empty)
30 | AnomalyDetectionExample.main(Array.empty)
31 | ConstraintSuggestionExample.main(Array.empty)
32 | }
33 | }
34 |
35 | }
36 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/metrics/MetricsTests.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.metrics
18 |
19 | import com.amazon.deequ.analyzers.DataTypeInstances
20 | import org.scalatest.{Matchers, WordSpec}
21 |
22 | import scala.util.{Failure, Success}
23 |
24 |
25 | class MetricsTests extends WordSpec with Matchers {
26 | val sampleException = new IllegalArgumentException()
27 | "Double metric" should {
28 | "flatten and return itself" in {
29 | val metric = DoubleMetric(Entity.Column, "metric-name", "instance-name", Success(50))
30 | assert(metric.flatten() == List(metric))
31 | }
32 |
33 | "flatten in case of an error" in {
34 | val metric = DoubleMetric(Entity.Column, "metric-name", "instance-name",
35 | Failure(sampleException))
36 | assert(metric.flatten() == List(metric))
37 | }
38 | }
39 |
40 | "Histogram metric" should {
41 | "flatten matched and unmatched" in {
42 | val distribution = Distribution(
43 | Map("a" -> DistributionValue(6, 0.6), "b" -> DistributionValue(4, 0.4)), 2)
44 |
45 | val metric = HistogramMetric("instance-name", Success(distribution))
46 |
47 | val expected = Seq(
48 | DoubleMetric(Entity.Column, "Histogram.bins", "instance-name", Success(2)),
49 | DoubleMetric(Entity.Column, "Histogram.abs.a", "instance-name", Success(6)),
50 | DoubleMetric(Entity.Column, "Histogram.abs.b", "instance-name", Success(4)),
51 | DoubleMetric(Entity.Column, "Histogram.ratio.a", "instance-name", Success(0.6)),
52 | DoubleMetric(Entity.Column, "Histogram.ratio.b", "instance-name", Success(0.4))
53 | ).toSet
54 | assert(metric.flatten().toSet == expected)
55 | }
56 |
57 | "flatten matched and unmatched in case of an error" in {
58 | val metric = HistogramMetric("instance-name", Failure(sampleException))
59 |
60 | val expected = Seq(DoubleMetric(Entity.Column, "Histogram.bins", "instance-name",
61 | Failure(sampleException))).toSet
62 | assert(metric.flatten().toSet == expected)
63 | }
64 | }
65 |
66 | }
67 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/package.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon
18 |
19 | import org.apache.spark.sql.{DataFrame, Row, SparkSession}
20 | import org.apache.spark.sql.types.{StructField, StructType}
21 | import org.apache.spark.sql.types.{ DataType => SparkDT }
22 |
23 | package object deequ {
24 | def dataFrameWithColumn(
25 | name: String,
26 | columnType: SparkDT,
27 | sparkSession: SparkSession,
28 | values: Row*)
29 | : DataFrame = {
30 |
31 | import scala.collection.JavaConverters._
32 | val struct = StructType(StructField(name, columnType) :: Nil)
33 | sparkSession.createDataFrame(values.asJava, struct).toDF(name)
34 | }
35 | }
36 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/suggestions/rules/interval/IntervalStrategyTest.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.suggestions.rules.interval
18 |
19 | import com.amazon.deequ.SparkContextSpec
20 | import com.amazon.deequ.suggestions.rules.interval.ConfidenceIntervalStrategy.ConfidenceInterval
21 | import com.amazon.deequ.utils.FixtureSupport
22 | import org.scalamock.scalatest.MockFactory
23 | import org.scalatest.Inspectors.forAll
24 | import org.scalatest.prop.Tables.Table
25 | import org.scalatest.wordspec.AnyWordSpec
26 |
27 | class IntervalStrategyTest extends AnyWordSpec with FixtureSupport with SparkContextSpec
28 | with MockFactory {
29 |
30 | "ConfidenceIntervalStrategy" should {
31 | "be calculated correctly" in {
32 | val waldStrategy = WaldIntervalStrategy()
33 | val wilsonStrategy = WilsonScoreIntervalStrategy()
34 |
35 | val table = Table(
36 | ("strategy", "pHat", "numRecord", "lowerBound", "upperBound"),
37 | (waldStrategy, 1.0, 20L, 1.0, 1.0),
38 | (waldStrategy, 0.5, 100L, 0.4, 0.6),
39 | (waldStrategy, 0.4, 100L, 0.3, 0.5),
40 | (waldStrategy, 0.6, 100L, 0.5, 0.7),
41 | (waldStrategy, 0.9, 100L, 0.84, 0.96),
42 | (waldStrategy, 1.0, 100L, 1.0, 1.0),
43 |
44 | (wilsonStrategy, 0.01, 20L, 0.00, 0.18),
45 | (wilsonStrategy, 1.0, 20L, 0.83, 1.0),
46 | (wilsonStrategy, 0.5, 100L, 0.4, 0.6),
47 | (wilsonStrategy, 0.4, 100L, 0.3, 0.5),
48 | (wilsonStrategy, 0.6, 100L, 0.5, 0.7),
49 | (wilsonStrategy, 0.9, 100L, 0.82, 0.95),
50 | (wilsonStrategy, 1.0, 100L, 0.96, 1.0)
51 | )
52 |
53 | forAll(table) { case (strategy, pHat, numRecords, lowerBound, upperBound) =>
54 | val actualInterval = strategy.calculateTargetConfidenceInterval(pHat, numRecords)
55 | assert(actualInterval == ConfidenceInterval(lowerBound, upperBound))
56 | }
57 | }
58 | }
59 | }
60 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/utils/AssertionUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.utils
18 |
19 | import scala.util.{Failure, Success, Try}
20 |
21 | object AssertionUtils {
22 |
23 | implicit class TryUtils[A](something: Try[A]) {
24 | def compare[B](other: Try[B]): Boolean = {
25 | (something, other) match {
26 | case (Success(a), Success(b)) => a == b
27 | case (Failure(a), Failure(b)) => a.getClass == b.getClass && (a.getMessage == b.getMessage)
28 | case (_, _) => false
29 | }
30 | }
31 | def compareFailureTypes[B](other: Try[B]): Boolean = {
32 | (something, other) match {
33 | case (Failure(a), Failure(b)) => a.getClass == b.getClass
34 | case (_, _) => false
35 | }
36 | }
37 | def compareOuterAndInnerFailureTypes[B](other: Try[B]): Boolean = {
38 | (something, other) match {
39 | case (Failure(a: Throwable), Failure(b: Throwable))
40 | if (a.getCause != null) && (b.getCause != null) =>
41 | (a.getClass == b.getClass) && (a.getCause.getClass == b.getCause.getClass)
42 | case (_, _) => false
43 | }
44 | }
45 |
46 | }
47 |
48 | }
49 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/utils/CollectionUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.utils
18 |
19 | object CollectionUtils {
20 |
21 | implicit class SeqExtensions[A](val source: Seq[A]) {
22 | def forEachOrder(f: Seq[A] => Any): Unit = {
23 | source.combinations(source.size)
24 | .flatMap { _.permutations }
25 | .foreach { distinctOrder => f(distinctOrder) }
26 | }
27 |
28 | }
29 | }
30 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/utils/ConditionUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2025 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.utils
18 |
19 | import software.amazon.glue.dqdl.model.condition.Condition
20 | import software.amazon.glue.dqdl.model.condition.number.{AtomicNumberOperand, NumberBasedCondition, NumberBasedConditionOperator, NumericOperand}
21 |
22 | import scala.collection.JavaConverters._
23 | import scala.util.matching.Regex
24 |
25 | object ConditionUtils {
26 | implicit class ConditionAsString(val expression: String) {
27 | private val dateRegex: Regex = "[0-9]{4}-[0-9]{2}-[0-9]{2}|now\\(\\)".r
28 |
29 | def convertOperand(op: String): NumericOperand = new AtomicNumberOperand(op)
30 |
31 | def asCondition: Condition = {
32 | expression match {
33 | case e if dateRegex.findFirstIn(e).isDefined => new Condition(e)
34 | case e if e.startsWith("between") =>
35 | val operands = e
36 | .replace("between", "")
37 | .split("and")
38 | .toList
39 | new NumberBasedCondition(expression,
40 | NumberBasedConditionOperator.BETWEEN, operands.map(convertOperand).asJava)
41 | case e if e.startsWith(">=") =>
42 | val operand = e.replace(">=", "")
43 | new NumberBasedCondition(expression,
44 | NumberBasedConditionOperator.GREATER_THAN_EQUAL_TO, List(convertOperand(operand)).asJava)
45 | case e if e.startsWith(">") =>
46 | val operand = e.replace(">", "")
47 | new NumberBasedCondition(expression,
48 | NumberBasedConditionOperator.GREATER_THAN, List(convertOperand(operand)).asJava)
49 | case e if e.startsWith("<=") =>
50 | val operand = e.replace("<=", "")
51 | new NumberBasedCondition(expression,
52 | NumberBasedConditionOperator.LESS_THAN_EQUAL_TO, List(convertOperand(operand)).asJava)
53 | case e if e.startsWith("<") =>
54 | val operand = e.replace("<", "")
55 | new NumberBasedCondition(expression,
56 | NumberBasedConditionOperator.LESS_THAN, List(convertOperand(operand)).asJava)
57 | case e if e.startsWith("=") =>
58 | val operand = e.replace("=", "")
59 | new NumberBasedCondition(expression,
60 | NumberBasedConditionOperator.EQUALS, List(convertOperand(operand)).asJava)
61 | case e => new Condition(e)
62 | }
63 | }
64 | }
65 | }
66 |
--------------------------------------------------------------------------------
/src/test/scala/com/amazon/deequ/utils/TempFileUtils.scala:
--------------------------------------------------------------------------------
1 | /**
2 | * Copyright 2018 Amazon.com, Inc. or its affiliates. All Rights Reserved.
3 | *
4 | * Licensed under the Apache License, Version 2.0 (the "License"). You may not
5 | * use this file except in compliance with the License. A copy of the License
6 | * is located at
7 | *
8 | * http://aws.amazon.com/apache2.0/
9 | *
10 | * or in the "license" file accompanying this file. This file is distributed on
11 | * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either
12 | * express or implied. See the License for the specific language governing
13 | * permissions and limitations under the License.
14 | *
15 | */
16 |
17 | package com.amazon.deequ.utils
18 |
19 | import java.nio.file.Files
20 | import java.util.UUID
21 |
22 | object TempFileUtils {
23 | def tempDir(prefix: String = UUID.randomUUID().toString): String = {
24 | val tempDir = Files.createTempDirectory(prefix).toFile
25 | tempDir.deleteOnExit()
26 | tempDir.getAbsolutePath
27 | }
28 | }
29 |
--------------------------------------------------------------------------------
/test-data/README.md:
--------------------------------------------------------------------------------
1 | # Dataset used for testing
2 |
3 | * [titanic.csv](https://www.kaggle.com/c/titanic/data)
--------------------------------------------------------------------------------