├── .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) --------------------------------------------------------------------------------