From 67e8dbbe971d6a30d9ab95d6c21045e1fa2965d1 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Sat, 24 May 2025 03:12:07 +0300 Subject: [PATCH 01/12] Profiler: add StringColumnProfile. --- docs/profiles.md | 1 + pydeequ/profiles.py | 38 ++++++++++++++++++++++++++++++++++++-- tests/test_profiles.py | 25 ++++++++++++++++++++++--- 3 files changed, 59 insertions(+), 5 deletions(-) diff --git a/docs/profiles.md b/docs/profiles.md index ee6b993..dfc0bbe 100644 --- a/docs/profiles.md +++ b/docs/profiles.md @@ -22,4 +22,5 @@ Here are the current supported functionalities of Profiles. | | property: profiles | Done | | | property: numRecords | Done | | StandardColumnProfile | StandardColumnProfile(spark_session, column, java_column_profile) | Done | +| StringColumnProfile | StringColumnProfile(spark_session, column, java_column_profile) | Done | | NumericColumnProfile | NumericColumnProfile(spark_session, column, java_column_profile) | Done | diff --git a/pydeequ/profiles.py b/pydeequ/profiles.py index 72ccc21..601257a 100644 --- a/pydeequ/profiles.py +++ b/pydeequ/profiles.py @@ -2,7 +2,9 @@ """ Profiles file for all the Profiles classes in Deequ""" import json from collections import namedtuple +from typing import Optional +from py4j.java_gateway import JavaObject from pyspark.sql import DataFrame, SparkSession from pydeequ.analyzers import KLLParameters from pydeequ.metrics import BucketDistribution @@ -241,9 +243,8 @@ def __init__(self, spark_session: SparkSession): self._numRecords = 0 self.columnProfileClasses = { "StandardColumnProfile": StandardColumnProfile, - "StringColumnProfile": StandardColumnProfile, + "StringColumnProfile": StringColumnProfile, "NumericColumnProfile": NumericColumnProfile, - } def _columnProfilesFromColumnRunBuilderRun(self, run): @@ -528,3 +529,36 @@ def approxPercentiles(self): """ return self._approxPercentiles + +class StringColumnProfile(StandardColumnProfile): + """ + String Column Profile class + + :param SparkSession spark_session: sparkSession + :param str column: the designated column of which the profile is run on + :param JavaObject java_column_profile: The profile mapped as a Java map + """ + + def __init__( + self, spark_session: SparkSession, column: str, java_column_profile: JavaObject + ) -> None: + super().__init__(spark_session, column, java_column_profile) + self._minLength = get_or_else_none(java_column_profile.minLength()) + self._maxLength = get_or_else_none(java_column_profile.maxLength()) + self.all.update( + { + "minLength": self._minLength, + "maxLength": self._maxLength, + } + ) + + @property + def minLength(self) -> Optional[int]: + return self._minLength + + @property + def maxLength(self) -> Optional[int]: + return self._maxLength + + def __str__(self) -> str: + return f"StringProfiles for column: {self.column}: {json.dumps(self.all, indent=4)}" diff --git a/tests/test_profiles.py b/tests/test_profiles.py index fd46a2c..3a28063 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -1,8 +1,7 @@ # -*- coding: utf-8 -*- import unittest from pyspark.sql import Row -from pydeequ.analyzers import KLLParameters -from pydeequ.profiles import ColumnProfilerRunBuilder, ColumnProfilerRunner +from pydeequ.profiles import ColumnProfilerRunBuilder, ColumnProfilerRunner, DistributionValue, StringColumnProfile from pydeequ.analyzers import KLLParameters, DataTypeInstances from tests.conftest import setup_pyspark @@ -11,7 +10,7 @@ class TestProfiles(unittest.TestCase): def setUpClass(cls): cls.spark = setup_pyspark().appName("test-profiles-local").getOrCreate() cls.sc = cls.spark.sparkContext - cls.df = cls.sc.parallelize([Row(a="foo", b=1, c=5), Row(a="bar", b=2, c=6), Row(a="baz", b=3, c=None)]).toDF() + cls.df = cls.sc.parallelize([Row(a="foo", b=1, c=5), Row(a="bar", b=2, c=6), Row(a="bazz", b=3, c=None)]).toDF() @classmethod def tearDownClass(cls): @@ -76,6 +75,26 @@ def test_profile_numRecords(self): result = ColumnProfilerRunner(self.spark).onData(self.df).run() self.assertEqual(result.numRecords, 3) + def test_StringColumnProfile(self): + result = ColumnProfilerRunner(self.spark).onData(self.df).run() + column_profile = result.profiles["a"] + self.assertIsInstance(column_profile, StringColumnProfile) + self.assertEqual(column_profile.minLength, 3) + self.assertEqual(column_profile.maxLength, 4) + + self.assertEqual(column_profile.completeness, 1.0) + self.assertEqual(column_profile.approximateNumDistinctValues, 3) + self.assertEqual(column_profile.typeCounts["String"], 3) + self.assertEqual(column_profile.isDataTypeInferred, False) + self.assertListEqual( + sorted(column_profile.histogram), + [ + DistributionValue("bar", 1, 1/3), + DistributionValue("bazz", 1, 1/3), + DistributionValue("foo", 1, 1/3), + ] + ) + if __name__ == "__main__": unittest.main() From 1d0fb2122885edabd79cd9b11375633d328ec6c1 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Mon, 4 May 2026 21:41:49 +0300 Subject: [PATCH 02/12] Profiler: remove JavaObject import used for typing purposes. --- pydeequ/profiles.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pydeequ/profiles.py b/pydeequ/profiles.py index 601257a..cd704f1 100644 --- a/pydeequ/profiles.py +++ b/pydeequ/profiles.py @@ -4,7 +4,6 @@ from collections import namedtuple from typing import Optional -from py4j.java_gateway import JavaObject from pyspark.sql import DataFrame, SparkSession from pydeequ.analyzers import KLLParameters from pydeequ.metrics import BucketDistribution @@ -540,7 +539,7 @@ class StringColumnProfile(StandardColumnProfile): """ def __init__( - self, spark_session: SparkSession, column: str, java_column_profile: JavaObject + self, spark_session: SparkSession, column: str, java_column_profile ) -> None: super().__init__(spark_session, column, java_column_profile) self._minLength = get_or_else_none(java_column_profile.minLength()) From d1cd8878726e9c0cbc0ad0b981eaad84addf6135 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Mon, 4 May 2026 21:54:17 +0300 Subject: [PATCH 03/12] Profiler: use assertAlmostEquals in test for float values, refine sorting. --- tests/test_profiles.py | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/tests/test_profiles.py b/tests/test_profiles.py index 3a28063..11739ab 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -86,14 +86,17 @@ def test_StringColumnProfile(self): self.assertEqual(column_profile.approximateNumDistinctValues, 3) self.assertEqual(column_profile.typeCounts["String"], 3) self.assertEqual(column_profile.isDataTypeInferred, False) - self.assertListEqual( - sorted(column_profile.histogram), - [ - DistributionValue("bar", 1, 1/3), - DistributionValue("bazz", 1, 1/3), - DistributionValue("foo", 1, 1/3), - ] - ) + actual_histogram = sorted(column_profile.histogram, key=lambda x: x.value) + self.assertEqual(len(actual_histogram), 3) + expected_histogram = [ + DistributionValue("bar", 1, 1 / 3), + DistributionValue("bazz", 1, 1 / 3), + DistributionValue("foo", 1, 1 / 3), + ] + for actual, expected in zip(actual_histogram, expected_histogram): + self.assertEqual(actual.value, expected.value) + self.assertEqual(actual.count, expected.count) + self.assertAlmostEquals(actual.ratio, expected.ratio) if __name__ == "__main__": From df371506ecfd28fee74ea25941a1ec6621028698 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Mon, 4 May 2026 22:05:56 +0300 Subject: [PATCH 04/12] Profiler: use assertAlmostEqual in test instead of assertAlmostEquals. --- tests/test_profiles.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/test_profiles.py b/tests/test_profiles.py index 11739ab..120a59b 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -96,7 +96,7 @@ def test_StringColumnProfile(self): for actual, expected in zip(actual_histogram, expected_histogram): self.assertEqual(actual.value, expected.value) self.assertEqual(actual.count, expected.count) - self.assertAlmostEquals(actual.ratio, expected.ratio) + self.assertAlmostEqual(actual.ratio, expected.ratio) if __name__ == "__main__": From f45f7122a8654607940d544e55a07cc1b877633d Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Thu, 7 May 2026 20:43:21 +0300 Subject: [PATCH 05/12] Profiler: add assertion for StringColumnProfile.__str__ result. --- tests/test_profiles.py | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/tests/test_profiles.py b/tests/test_profiles.py index 120a59b..9e35121 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -81,6 +81,43 @@ def test_StringColumnProfile(self): self.assertIsInstance(column_profile, StringColumnProfile) self.assertEqual(column_profile.minLength, 3) self.assertEqual(column_profile.maxLength, 4) + self.assertEqual( + str(column_profile), + ( + 'StringProfiles for column: a: {\n' + ' "completeness": 1.0,\n' + ' "approximateNumDistinctValues": 3,\n' + ' "dataType": "String",\n' + ' "isDataTypeInferred": false,\n' + ' "typeCounts": {\n' + ' "Boolean": 0,\n' + ' "Fractional": 0,\n' + ' "Integral": 0,\n' + ' "Unknown": 0,\n' + ' "String": 3\n' + ' },\n' + ' "histogram": [\n' + ' [\n' + ' "bazz",\n' + ' 1,\n' + ' 0.3333333333333333\n' + ' ],\n' + ' [\n' + ' "foo",\n' + ' 1,\n' + ' 0.3333333333333333\n' + ' ],\n' + ' [\n' + ' "bar",\n' + ' 1,\n' + ' 0.3333333333333333\n' + ' ]\n' + ' ],\n' + ' "minLength": 3,\n' + ' "maxLength": 4\n' + '}' + ) + ) self.assertEqual(column_profile.completeness, 1.0) self.assertEqual(column_profile.approximateNumDistinctValues, 3) From c961d0b2962ee757927b087657ec6ed222172ed0 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Fri, 8 May 2026 01:05:22 +0300 Subject: [PATCH 06/12] StringColumnProfile: build self.all from scratch instead of updating parent's. --- pydeequ/profiles.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/pydeequ/profiles.py b/pydeequ/profiles.py index cd704f1..978fee9 100644 --- a/pydeequ/profiles.py +++ b/pydeequ/profiles.py @@ -544,12 +544,16 @@ def __init__( super().__init__(spark_session, column, java_column_profile) self._minLength = get_or_else_none(java_column_profile.minLength()) self._maxLength = get_or_else_none(java_column_profile.maxLength()) - self.all.update( - { - "minLength": self._minLength, - "maxLength": self._maxLength, - } - ) + self.all = { + "completeness": self.completeness, + "approximateNumDistinctValues": self.approximateNumDistinctValues, + "dataType": self.dataType, + "isDataTypeInferred": self.isDataTypeInferred, + "typeCounts": self.typeCounts, + "histogram": self.histogram, + "minLength": self._minLength, + "maxLength": self._maxLength, + } @property def minLength(self) -> Optional[int]: From 3ad6cd9528952e8978f545d0950bbe68ffe7d8ba Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Fri, 8 May 2026 01:24:21 +0300 Subject: [PATCH 07/12] StringColumnProfile: shorten __str__ result assertion. --- tests/test_profiles.py | 38 +------------------------------------- 1 file changed, 1 insertion(+), 37 deletions(-) diff --git a/tests/test_profiles.py b/tests/test_profiles.py index 9e35121..b45750d 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -81,43 +81,7 @@ def test_StringColumnProfile(self): self.assertIsInstance(column_profile, StringColumnProfile) self.assertEqual(column_profile.minLength, 3) self.assertEqual(column_profile.maxLength, 4) - self.assertEqual( - str(column_profile), - ( - 'StringProfiles for column: a: {\n' - ' "completeness": 1.0,\n' - ' "approximateNumDistinctValues": 3,\n' - ' "dataType": "String",\n' - ' "isDataTypeInferred": false,\n' - ' "typeCounts": {\n' - ' "Boolean": 0,\n' - ' "Fractional": 0,\n' - ' "Integral": 0,\n' - ' "Unknown": 0,\n' - ' "String": 3\n' - ' },\n' - ' "histogram": [\n' - ' [\n' - ' "bazz",\n' - ' 1,\n' - ' 0.3333333333333333\n' - ' ],\n' - ' [\n' - ' "foo",\n' - ' 1,\n' - ' 0.3333333333333333\n' - ' ],\n' - ' [\n' - ' "bar",\n' - ' 1,\n' - ' 0.3333333333333333\n' - ' ]\n' - ' ],\n' - ' "minLength": 3,\n' - ' "maxLength": 4\n' - '}' - ) - ) + self.assertEqual(str(column_profile)[0:29], "StringProfiles for column: a:") self.assertEqual(column_profile.completeness, 1.0) self.assertEqual(column_profile.approximateNumDistinctValues, 3) From a81ee33d85b14cbb11b7d48f7a58b01c9130bf63 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Fri, 8 May 2026 01:55:47 +0300 Subject: [PATCH 08/12] StringColumnProfile: add assertion for __str__ result. --- tests/test_profiles.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/test_profiles.py b/tests/test_profiles.py index b45750d..d3007cf 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -82,6 +82,7 @@ def test_StringColumnProfile(self): self.assertEqual(column_profile.minLength, 3) self.assertEqual(column_profile.maxLength, 4) self.assertEqual(str(column_profile)[0:29], "StringProfiles for column: a:") + self.assertIn('"minLength": 3', str(column_profile)) self.assertEqual(column_profile.completeness, 1.0) self.assertEqual(column_profile.approximateNumDistinctValues, 3) From 24ee7674a8386b801baa081b1ff6686413c41a72 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Fri, 8 May 2026 10:08:27 +0300 Subject: [PATCH 09/12] StringColumProfile: use dedicated df for testing. --- tests/test_profiles.py | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/tests/test_profiles.py b/tests/test_profiles.py index d3007cf..b09e652 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -10,7 +10,7 @@ class TestProfiles(unittest.TestCase): def setUpClass(cls): cls.spark = setup_pyspark().appName("test-profiles-local").getOrCreate() cls.sc = cls.spark.sparkContext - cls.df = cls.sc.parallelize([Row(a="foo", b=1, c=5), Row(a="bar", b=2, c=6), Row(a="bazz", b=3, c=None)]).toDF() + cls.df = cls.sc.parallelize([Row(a="foo", b=1, c=5), Row(a="bar", b=2, c=6), Row(a="baz", b=3, c=None)]).toDF() @classmethod def tearDownClass(cls): @@ -76,24 +76,25 @@ def test_profile_numRecords(self): self.assertEqual(result.numRecords, 3) def test_StringColumnProfile(self): - result = ColumnProfilerRunner(self.spark).onData(self.df).run() + df = self.sc.parallelize([Row(a="ant"), Row(a="bee"), Row(a="bee"), Row(a="cricket")]).toDF() + result = ColumnProfilerRunner(self.spark).onData(df).run() column_profile = result.profiles["a"] self.assertIsInstance(column_profile, StringColumnProfile) self.assertEqual(column_profile.minLength, 3) - self.assertEqual(column_profile.maxLength, 4) + self.assertEqual(column_profile.maxLength, 7) self.assertEqual(str(column_profile)[0:29], "StringProfiles for column: a:") self.assertIn('"minLength": 3', str(column_profile)) - self.assertEqual(column_profile.completeness, 1.0) + self.assertEqual(column_profile.completeness, 1) self.assertEqual(column_profile.approximateNumDistinctValues, 3) - self.assertEqual(column_profile.typeCounts["String"], 3) + self.assertEqual(column_profile.typeCounts["String"], 4) self.assertEqual(column_profile.isDataTypeInferred, False) actual_histogram = sorted(column_profile.histogram, key=lambda x: x.value) self.assertEqual(len(actual_histogram), 3) expected_histogram = [ - DistributionValue("bar", 1, 1 / 3), - DistributionValue("bazz", 1, 1 / 3), - DistributionValue("foo", 1, 1 / 3), + DistributionValue("ant", 1, 0.25), + DistributionValue("bee", 2, 0.5), + DistributionValue("cricket", 1, 0.25), ] for actual, expected in zip(actual_histogram, expected_histogram): self.assertEqual(actual.value, expected.value) From 9c29f0d88aa22edca4e37ddd50b626c5af2724a7 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Fri, 8 May 2026 10:09:11 +0300 Subject: [PATCH 10/12] Profile tests: apply black formatting. --- tests/test_profiles.py | 26 +++++++++++++++++++++----- 1 file changed, 21 insertions(+), 5 deletions(-) diff --git a/tests/test_profiles.py b/tests/test_profiles.py index b09e652..4268875 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -1,16 +1,24 @@ # -*- coding: utf-8 -*- import unittest from pyspark.sql import Row -from pydeequ.profiles import ColumnProfilerRunBuilder, ColumnProfilerRunner, DistributionValue, StringColumnProfile +from pydeequ.profiles import ( + ColumnProfilerRunBuilder, + ColumnProfilerRunner, + DistributionValue, + StringColumnProfile, +) from pydeequ.analyzers import KLLParameters, DataTypeInstances from tests.conftest import setup_pyspark + class TestProfiles(unittest.TestCase): @classmethod def setUpClass(cls): cls.spark = setup_pyspark().appName("test-profiles-local").getOrCreate() cls.sc = cls.spark.sparkContext - cls.df = cls.sc.parallelize([Row(a="foo", b=1, c=5), Row(a="bar", b=2, c=6), Row(a="baz", b=3, c=None)]).toDF() + cls.df = cls.sc.parallelize( + [Row(a="foo", b=1, c=5), Row(a="bar", b=2, c=6), Row(a="baz", b=3, c=None)] + ).toDF() @classmethod def tearDownClass(cls): @@ -18,10 +26,18 @@ def tearDownClass(cls): cls.spark.stop() def test_setPredefinedTypes(self): - result = ColumnProfilerRunner(self.spark) \ - .onData(self.df) \ - .setPredefinedTypes({'a': DataTypeInstances.Unknown, 'b': DataTypeInstances.String, 'c': DataTypeInstances.Fractional}) \ + result = ( + ColumnProfilerRunner(self.spark) + .onData(self.df) + .setPredefinedTypes( + { + "a": DataTypeInstances.Unknown, + "b": DataTypeInstances.String, + "c": DataTypeInstances.Fractional, + } + ) .run() + ) print(result) for col, profile in result.profiles.items(): print("Profiles:", profile) From 4e45707c4798cc573c9d0b0e547d87b141b0aa36 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Fri, 8 May 2026 10:17:07 +0300 Subject: [PATCH 11/12] StringColumProfile: refactor to subclass ColumnProfile. --- pydeequ/profiles.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pydeequ/profiles.py b/pydeequ/profiles.py index 978fee9..c0f4c6c 100644 --- a/pydeequ/profiles.py +++ b/pydeequ/profiles.py @@ -529,7 +529,7 @@ def approxPercentiles(self): return self._approxPercentiles -class StringColumnProfile(StandardColumnProfile): +class StringColumnProfile(ColumnProfile): """ String Column Profile class From ea04bca0bea0235d288a99466ec36c8c49345c58 Mon Sep 17 00:00:00 2001 From: Eugene Nikolayev Date: Fri, 8 May 2026 10:31:17 +0300 Subject: [PATCH 12/12] StringColumProfile: add tests for None values --- tests/test_profiles.py | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/tests/test_profiles.py b/tests/test_profiles.py index 4268875..9ac45c2 100644 --- a/tests/test_profiles.py +++ b/tests/test_profiles.py @@ -92,7 +92,14 @@ def test_profile_numRecords(self): self.assertEqual(result.numRecords, 3) def test_StringColumnProfile(self): - df = self.sc.parallelize([Row(a="ant"), Row(a="bee"), Row(a="bee"), Row(a="cricket")]).toDF() + df = self.sc.parallelize( + [ + Row(a="ant", b="dragonfly"), + Row(a="bee", b="earwig"), + Row(a="bee", b=None), + Row(a="cricket", b=None), + ] + ).toDF() result = ColumnProfilerRunner(self.spark).onData(df).run() column_profile = result.profiles["a"] self.assertIsInstance(column_profile, StringColumnProfile) @@ -117,6 +124,13 @@ def test_StringColumnProfile(self): self.assertEqual(actual.count, expected.count) self.assertAlmostEqual(actual.ratio, expected.ratio) + column_profile = result.profiles["b"] + self.assertEqual(column_profile.completeness, 0.5) + self.assertEqual(column_profile.approximateNumDistinctValues, 2) + self.assertEqual(column_profile.typeCounts["String"], 2) + self.assertEqual(column_profile.minLength, 0) + self.assertEqual(column_profile.maxLength, 9) + if __name__ == "__main__": unittest.main()