From 738d2207f965ae5f6e888bda0aa5f357c2dad8f4 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Wed, 19 Oct 2022 20:06:58 -0700 Subject: [PATCH 01/26] wip --- dbldatagen/data_generator.py | 57 +++++++++++++++++---------- tests/test_streaming.py | 76 ++++++++++++++++++++++++++++++++++-- 2 files changed, 110 insertions(+), 23 deletions(-) diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index 0ae8d396..c98a397a 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -778,29 +778,46 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): if ColumnGenerationSpec.SEED_COLUMN != "id": df1 = df1.withColumnRenamed("id", ColumnGenerationSpec.SEED_COLUMN) + else: + df1 = self._getStreamingBaseDataFrame(startId, options) + + return df1 + def _getStreamingBaseDataFrame(self, startId=0, options=None): + end_id = self._rowCount + startId + id_partitions = (self.partitions if self.partitions is not None + else self.sparkSession.sparkContext.defaultParallelism) + + status = f"Generating streaming data frame with ids from {startId} to {end_id} with {id_partitions} partitions" + self.logger.info(status) + self.executionHistory.append(status) + + df1 = (self.sparkSession.readStream + .format("rate")) + if options is not None: + if "rowsPerSecond" not in options: + options['rowsPerSecond'] = 1 + if "numPartitions" not in options: + options['numPartitions'] = id_partitions else: - status = (f"Generating streaming data frame with ids from {startId} to {end_id} with {id_partitions} partitions") - self.logger.info(status) - self.executionHistory.append(status) + options = { + "rowsPerSecond": 1, + "numPartitions": id_partitions + } - df1 = (self.sparkSession.readStream - .format("rate")) - if options is not None: - if "rowsPerSecond" not in options: - options['rowsPerSecond'] = 1 - if "numPartitions" not in options: - options['numPartitions'] = id_partitions - - for k, v in options.items(): - df1 = df1.option(k, v) - df1 = df1.load().withColumnRenamed("value", ColumnGenerationSpec.SEED_COLUMN) - else: - df1 = (df1.option("rowsPerSecond", 1) - .option("numPartitions", id_partitions) - .load() - .withColumnRenamed("value", ColumnGenerationSpec.SEED_COLUMN) - ) + age_limit_interval = None + + if "ageLimit" in options: + age_limit_interval = options.pop("ageLimit") + assert age_limit_interval is not None and float(age_limit_interval) > 0.0, "invalid age limit" + + for k, v in options.items(): + df1 = df1.option(k, v) + df1 = df1.load().withColumnRenamed("value", ColumnGenerationSpec.SEED_COLUMN) + + if age_limit_interval is not None: + df1 = df1.where(f"""abs(cast(now() as double) - cast(`timestamp` as double )) + < cast({age_limit_interval} as double)""") return df1 diff --git a/tests/test_streaming.py b/tests/test_streaming.py index a4a51b39..9a99ccd2 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -13,14 +13,14 @@ class TestStreaming(unittest.TestCase): row_count = 100000 column_count = 10 - time_to_run = 15 + time_to_run = 8 rows_per_second = 5000 def test_streaming(self): time_now = int(round(time.time() * 1000)) base_dir = "/tmp/testdatagenerator_{}".format(time_now) - test_dir = os.path.join(base_dir, "data") - checkpoint_dir = os.path.join(base_dir, "checkpoint") + test_dir = os.path.join(base_dir, "data1") + checkpoint_dir = os.path.join(base_dir, "checkpoint1") print(time_now, test_dir, checkpoint_dir) new_data_rows = 0 @@ -80,3 +80,73 @@ def test_streaming(self): # check that we have at least one second of data self.assertGreater(new_data_rows, self.rows_per_second) + + def test_streaming_with_age_limit(self): + print(spark.version) + + time_now = int(round(time.time() * 1000)) + base_dir = "/tmp/testdatagenerator2_{}".format(time_now) + test_dir = os.path.join(base_dir, "data") + checkpoint_dir = os.path.join(base_dir, "checkpoint") + print(time_now, test_dir, checkpoint_dir) + + new_data_rows = 0 + + try: + os.makedirs(test_dir) + os.makedirs(checkpoint_dir) + + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, + partitions=4, seedMethod='hash_fieldname') + .withIdOutput() + .withColumn("r", FloatType(), expr="floor(rand() * 350) * (86400 + 3600)", + numColumns=self.column_count) + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code2", IntegerType(), minValue=0, maxValue=10) + .withColumn("code3", StringType(), values=['a', 'b', 'c']) + .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + + ) + + dfTestData = testDataSpec.build(withStreaming=True, + options={'rowsPerSecond': self.rows_per_second, + 'ageLimit': 1}) + + (dfTestData + .writeStream + .format("parquet") + .outputMode("append") + .option("path", test_dir) + .option("checkpointLocation", checkpoint_dir) + .start()) + + start_time = time.time() + time.sleep(self.time_to_run) + + # note stopping the stream may produce exceptions - these can be ignored + recent_progress = [] + for x in spark.streams.active: + recent_progress.append(x.recentProgress) + print(x) + x.stop() + + end_time = time.time() + + # read newly written data + df2 = spark.read.format("parquet").load(test_dir) + + new_data_rows = df2.count() + + print("read {} rows from newly written data".format(new_data_rows)) + finally: + shutil.rmtree(base_dir) + + print("*** Done ***") + + print("elapsed time (seconds)", end_time - start_time) + + # check that we have at least one second of data + self.assertGreater(new_data_rows, int(self.rows_per_second / 4)) + + From 94a90e839f69dbee790b77d5dab0ca6e1cb5ae04 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Wed, 19 Oct 2022 22:31:58 -0700 Subject: [PATCH 02/26] wip --- dbldatagen/data_generator.py | 96 ++++++++++++++++++++++++++++++------ 1 file changed, 82 insertions(+), 14 deletions(-) diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index c98a397a..493deb10 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -16,8 +16,18 @@ from .spark_singleton import SparkSingleton from .utils import ensure, topologicalSort, DataGenError, deprecated +START_TIMESTAMP_OPTION = "startTimestamp" +ROWS_PER_SECOND_OPTION = "rowsPerSecond" +AGE_LIMIT_OPTION = "ageLimit" +NUM_PARTITIONS_OPTION = "numPartitions" +ROWS_PER_BATCH_OPTION = "rowsPerBatch" +STREAMING_SOURCE_OPTION = "streamingSource" + _OLD_MIN_OPTION = 'min' _OLD_MAX_OPTION = 'max' +RATE_SOURCE = "rate" +RATE_PER_MICRO_BATCH_SOURCE = "rate-micro-batch" +SPARK_RATE_MICROBATCH_VERSION = "3.2.1" class DataGenerator: @@ -783,36 +793,94 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): return df1 + def _getStreamingSource(self, options=None): + """ get streaming source from options + + :param options: dictionary of options + :returns: streaming source if present in options (popping option from options), or default if not present + + Default streaming source is computed based on whether we are running on Spark version 3.2.1 or later + + if using spark version 3.2.1 or later - `rate-micro-batch` is used as source, otherwise `rate` is used as source + """ + if options is not None: + if STREAMING_SOURCE_OPTION in options: + streaming_source = options.pop(STREAMING_SOURCE_OPTION) + assert streaming_source in [RATE_SOURCE, RATE_PER_MICRO_BATCH_SOURCE], \ + f"Invalid streaming source - only ['{RATE_SOURCE}', ['{RATE_PER_MICRO_BATCH_SOURCE}'] supported" + + # if using Spark 3.2.1, then default should be RATE_PER_MICRO_BATCH_SOURCE + if self.sparkSession.version >= SPARK_RATE_MICROBATCH_VERSION: + streaming_source = RATE_PER_MICRO_BATCH_SOURCE + else: + streaming_source = RATE_SOURCE + + return streaming_source + + def _getCurrentSparkTimestamp(self, asLong=False): + """ get current spark timestamp + + :param asLong: if True, returns current spark timestamp as long, string otherwise + """ + if asLong: + return (self.sparkSession.sql(f"select cast(now() as string) as start_timestamp") + .collect()[0]['start_timestamp']) + else: + return (self.sparkSession.sql(f"select cast(now() as long) as start_timestamp") + .collect()[0]['start_timestamp']) + def _getStreamingBaseDataFrame(self, startId=0, options=None): + """Generate base streaming data frame""" end_id = self._rowCount + startId id_partitions = (self.partitions if self.partitions is not None else self.sparkSession.sparkContext.defaultParallelism) - status = f"Generating streaming data frame with ids from {startId} to {end_id} with {id_partitions} partitions" + # determine streaming source + streaming_source = self._getStreamingSource(options) + + if options is None: + if streaming_source == RATE_SOURCE: + options = { + ROWS_PER_SECOND_OPTION: 1, + } + else: + options = { + ROWS_PER_BATCH_OPTION: 1, + } + + if NUM_PARTITIONS_OPTION not in options: + options[NUM_PARTITIONS_OPTION] = id_partitions + + if streaming_source == RATE_PER_MICRO_BATCH_SOURCE: + if START_TIMESTAMP_OPTION not in options: + options[ START_TIMESTAMP_OPTION] = self._getCurrentSparkTimestamp(asLong=True) + + if ROWS_PER_BATCH_OPTION not in options: + options[ ROWS_PER_BATCH_OPTION] = id_partitions + status = f"Generating streaming data from rate source with {id_partitions} partitions" + + elif streaming_source == RATE_SOURCE: + if ROWS_PER_SECOND_OPTION not in options: + options[ ROWS_PER_SECOND_OPTION] = 1 + status = f"Generating streaming data from rate-micro-batch source with {id_partitions} partitions" + else: + assert streaming_source in [RATE_SOURCE, RATE_PER_MICRO_BATCH_SOURCE], \ + f"Invalid streaming source - only ['{RATE_SOURCE}', ['{RATE_PER_MICRO_BATCH_SOURCE}'] supported" + self.logger.info(status) self.executionHistory.append(status) - df1 = (self.sparkSession.readStream - .format("rate")) - if options is not None: - if "rowsPerSecond" not in options: - options['rowsPerSecond'] = 1 - if "numPartitions" not in options: - options['numPartitions'] = id_partitions - else: - options = { - "rowsPerSecond": 1, - "numPartitions": id_partitions - } + df1 = (self.sparkSession.readStream.format(streaming_source)) age_limit_interval = None - if "ageLimit" in options: + if AGE_LIMIT_OPTION in options: age_limit_interval = options.pop("ageLimit") assert age_limit_interval is not None and float(age_limit_interval) > 0.0, "invalid age limit" for k, v in options.items(): df1 = df1.option(k, v) + df1 = df1.load().withColumnRenamed("value", ColumnGenerationSpec.SEED_COLUMN) if age_limit_interval is not None: From 51adf3968b392662b8dd52054849c8d6a9c56f15 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Thu, 20 Oct 2022 02:34:15 -0700 Subject: [PATCH 03/26] wip --- dbldatagen/data_generator.py | 99 ++++++---- makefile | 8 + python/dev_require_321.txt | 33 ++++ tests/test_streaming.py | 346 ++++++++++++++++++++++++++++++++--- 4 files changed, 422 insertions(+), 64 deletions(-) create mode 100644 python/dev_require_321.txt diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index 493deb10..7dc9a0d8 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -267,7 +267,7 @@ def explain(self, suppressOutput=False): output = ["", "Data generation plan", "====================", f"spec=DateGenerator(name={self.name}, rows={self._rowCount}, startingId={self.starting_id}, partitions={self.partitions})" - , ")", "", f"column build order: {self._buildOrder}", "", "build plan:"] + , ")", "", f"column build order: {self._buildOrder}", "", "build plan:"] for plan_action in self._buildPlan: output.append(" ==> " + plan_action) @@ -793,7 +793,7 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): return df1 - def _getStreamingSource(self, options=None): + def _getStreamingSource(self, options=None, spark_version=None): """ get streaming source from options :param options: dictionary of options @@ -803,17 +803,22 @@ def _getStreamingSource(self, options=None): if using spark version 3.2.1 or later - `rate-micro-batch` is used as source, otherwise `rate` is used as source """ + streaming_source = None if options is not None: if STREAMING_SOURCE_OPTION in options: - streaming_source = options.pop(STREAMING_SOURCE_OPTION) + streaming_source = options[STREAMING_SOURCE_OPTION] assert streaming_source in [RATE_SOURCE, RATE_PER_MICRO_BATCH_SOURCE], \ f"Invalid streaming source - only ['{RATE_SOURCE}', ['{RATE_PER_MICRO_BATCH_SOURCE}'] supported" - # if using Spark 3.2.1, then default should be RATE_PER_MICRO_BATCH_SOURCE - if self.sparkSession.version >= SPARK_RATE_MICROBATCH_VERSION: - streaming_source = RATE_PER_MICRO_BATCH_SOURCE - else: - streaming_source = RATE_SOURCE + if spark_version is None: + spark_version = self.sparkSession.version + + if streaming_source is None: + # if using Spark 3.2.1, then default should be RATE_PER_MICRO_BATCH_SOURCE + if spark_version >= SPARK_RATE_MICROBATCH_VERSION: + streaming_source = RATE_PER_MICRO_BATCH_SOURCE + else: + streaming_source = RATE_SOURCE return streaming_source @@ -823,61 +828,77 @@ def _getCurrentSparkTimestamp(self, asLong=False): :param asLong: if True, returns current spark timestamp as long, string otherwise """ if asLong: - return (self.sparkSession.sql(f"select cast(now() as string) as start_timestamp") - .collect()[0]['start_timestamp']) - else: return (self.sparkSession.sql(f"select cast(now() as long) as start_timestamp") - .collect()[0]['start_timestamp']) + .collect()[0]['start_timestamp']) + else: + return (self.sparkSession.sql(f"select cast(now() as string) as start_timestamp") + .collect()[0]['start_timestamp']) - def _getStreamingBaseDataFrame(self, startId=0, options=None): - """Generate base streaming data frame""" - end_id = self._rowCount + startId - id_partitions = (self.partitions if self.partitions is not None - else self.sparkSession.sparkContext.defaultParallelism) + def _prepareStreamingOptions(self, options=None, spark_version=None): + default_streaming_partitions = (self.partitions if self.partitions is not None + else self.sparkSession.sparkContext.defaultParallelism) - # determine streaming source - streaming_source = self._getStreamingSource(options) + streaming_source = self._getStreamingSource(options, spark_version) if options is None: - if streaming_source == RATE_SOURCE: - options = { - ROWS_PER_SECOND_OPTION: 1, - } - else: - options = { - ROWS_PER_BATCH_OPTION: 1, - } + new_options = ({ROWS_PER_SECOND_OPTION: default_streaming_partitions} if streaming_source == RATE_SOURCE + else {ROWS_PER_BATCH_OPTION: default_streaming_partitions}) + else: + new_options = options.copy() - if NUM_PARTITIONS_OPTION not in options: - options[NUM_PARTITIONS_OPTION] = id_partitions + if NUM_PARTITIONS_OPTION in new_options: + streaming_partitions = new_options[NUM_PARTITIONS_OPTION] + else: + streaming_partitions = default_streaming_partitions + new_options[NUM_PARTITIONS_OPTION] = streaming_partitions if streaming_source == RATE_PER_MICRO_BATCH_SOURCE: - if START_TIMESTAMP_OPTION not in options: - options[ START_TIMESTAMP_OPTION] = self._getCurrentSparkTimestamp(asLong=True) + if START_TIMESTAMP_OPTION not in new_options: + new_options[START_TIMESTAMP_OPTION] = self._getCurrentSparkTimestamp(asLong=True) - if ROWS_PER_BATCH_OPTION not in options: - options[ ROWS_PER_BATCH_OPTION] = id_partitions - status = f"Generating streaming data from rate source with {id_partitions} partitions" + if ROWS_PER_BATCH_OPTION not in new_options: + # generate one row per partition + new_options[ROWS_PER_BATCH_OPTION] = streaming_partitions elif streaming_source == RATE_SOURCE: - if ROWS_PER_SECOND_OPTION not in options: - options[ ROWS_PER_SECOND_OPTION] = 1 - status = f"Generating streaming data from rate-micro-batch source with {id_partitions} partitions" + if ROWS_PER_SECOND_OPTION not in new_options: + new_options[ROWS_PER_SECOND_OPTION] = streaming_partitions else: assert streaming_source in [RATE_SOURCE, RATE_PER_MICRO_BATCH_SOURCE], \ f"Invalid streaming source - only ['{RATE_SOURCE}', ['{RATE_PER_MICRO_BATCH_SOURCE}'] supported" + return streaming_source, new_options + + def _getStreamingBaseDataFrame(self, startId=0, options=None): + """Generate base streaming data frame""" + end_id = self._rowCount + startId + + # determine streaming source + streaming_source, options = self._prepareStreamingOptions(options) + partitions = options[NUM_PARTITIONS_OPTION] + + if streaming_source == RATE_SOURCE: + status = f"Generating streaming data with rate source with {partitions} partitions" + else: + status = f"Generating streaming data with rate-micro-batch source with {partitions} partitions" + self.logger.info(status) self.executionHistory.append(status) - df1 = (self.sparkSession.readStream.format(streaming_source)) - age_limit_interval = None + if STREAMING_SOURCE_OPTION in options: + options.pop(STREAMING_SOURCE_OPTION) + if AGE_LIMIT_OPTION in options: age_limit_interval = options.pop("ageLimit") assert age_limit_interval is not None and float(age_limit_interval) > 0.0, "invalid age limit" + assert AGE_LIMIT_OPTION not in options + assert STREAMING_SOURCE_OPTION not in options + + df1 = self.sparkSession.readStream.format(streaming_source) + for k, v in options.items(): df1 = df1.option(k, v) diff --git a/makefile b/makefile index 953eab87..9cf158cc 100644 --- a/makefile +++ b/makefile @@ -29,6 +29,10 @@ create-dev-env: @echo "$(OK_COLOR)=> making conda dev environment$(NO_COLOR)" conda create -n $(ENV_NAME) python=3.7.5 +create-dev-env-321: + @echo "$(OK_COLOR)=> making conda dev environment for Spark 3.2.1$(NO_COLOR)" + conda create -n $(ENV_NAME) python=3.8.10 + create-github-build-env: @echo "$(OK_COLOR)=> making conda dev environment$(NO_COLOR)" conda create -n pip_$(ENV_NAME) python=3.8 @@ -37,6 +41,10 @@ install-dev-dependencies: @echo "$(OK_COLOR)=> installing dev environment requirements$(NO_COLOR)" pip install -r python/dev_require.txt +install-dev-dependencies321: + @echo "$(OK_COLOR)=> installing dev environment requirements for Spark 3.2.1$(NO_COLOR)" + pip install -r python/dev_require_321.txt + clean-dev-env: @echo "$(OK_COLOR)=> Cleaning dev environment$(NO_COLOR)" @echo "Current version: $(CURRENT_VERSION)" diff --git a/python/dev_require_321.txt b/python/dev_require_321.txt new file mode 100644 index 00000000..56846a62 --- /dev/null +++ b/python/dev_require_321.txt @@ -0,0 +1,33 @@ +# The following packages are used in building the test data generator framework. +# All packages used are already installed in the Databricks runtime environment for version 6.5 or later +numpy==1.20.1 +pandas==1.2.4 +pickleshare==0.7.5 +py4j==0.10.9.3 +pyarrow==4.0.0 +pyspark==3.2.1 +python-dateutil==2.8.1 +six==1.15.0 + +# The following packages are required for development only +wheel==0.36.2 +setuptools==52.0.0 +bumpversion +pytest +pytest-cov +pytest-timeout +rstcheck +prospector + +# The following packages are only required for building documentation and are not required at runtime +sphinx==5.0.0 +sphinx_rtd_theme +nbsphinx +numpydoc==0.8 +pypandoc +ipython==7.16.3 +recommonmark +sphinx-markdown-builder +rst2pdf==0.98 +Jinja2 < 3.1 + diff --git a/tests/test_streaming.py b/tests/test_streaming.py index 9a99ccd2..92ac2bed 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -1,7 +1,7 @@ import os import shutil import time -import unittest +import pytest from pyspark.sql.types import IntegerType, StringType, FloatType @@ -10,48 +10,344 @@ spark = dg.SparkSingleton.getLocalInstance("streaming tests") -class TestStreaming(unittest.TestCase): +class TestStreaming: row_count = 100000 column_count = 10 time_to_run = 8 rows_per_second = 5000 - def test_streaming(self): + def getTestDataSpec(self): + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", + rows=self.row_count, + partitions=spark.sparkContext.defaultParallelism, + seedMethod='hash_fieldname') + .withIdOutput() + .withColumn("r", FloatType(), expr="floor(rand() * 350) * (86400 + 3600)", + numColumns=self.column_count) + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code2", IntegerType(), minValue=0, maxValue=10) + .withColumn("code3", StringType(), values=['a', 'b', 'c']) + .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + ) + return testDataSpec + + def test_get_current_spark_timestamp(self): + testDataSpec = dg.DataGenerator(sparkSession=spark, name="test_data_set1", + rows=self.row_count, + partitions=spark.sparkContext.defaultParallelism, + seedMethod='hash_fieldname') + ts = testDataSpec._getCurrentSparkTimestamp(asLong=False) + + assert type(ts) is str + assert ts is not None and len(ts.strip()) > 0 + print(ts) + + def test_get_current_spark_timestamp2(self): + testDataSpec = dg.DataGenerator(sparkSession=spark, name="test_data_set1", + rows=self.row_count, + partitions=spark.sparkContext.defaultParallelism, + seedMethod='hash_fieldname') + ts = testDataSpec._getCurrentSparkTimestamp(asLong=True) + + assert(type(ts) is int) + print(ts) + + def test_get_current_spark_version(self): + assert spark.version > "3.0.0" + assert spark.version <= "6.0.0" + + @pytest.mark.parametrize("options_supplied,expected,spark_version_override", + [(None, "rate" if spark.version < "3.2.1" else "rate-micro-batch", None), + (None, "rate", "3.0.0"), + (None, "rate-micro-batch", "3.2.1"), + ({'streamingSource': 'rate'}, 'rate', None), + ({'streamingSource': 'rate-micro-batch'}, 'rate-micro-batch', None)]) + def test_streaming_source_options(self, options_supplied, expected, spark_version_override): + print("options", options_supplied) + testDataSpec = dg.DataGenerator(sparkSession=spark, name="test_data_set1", + rows=self.row_count, + partitions=spark.sparkContext.defaultParallelism, + seedMethod='hash_fieldname') + + result = testDataSpec._getStreamingSource(options_supplied, spark_version_override) + print("Options:", options_supplied, "retval:", result) + + assert result == expected + + @pytest.mark.parametrize("options_supplied,source_expected,options_expected,spark_version_override", + [(None, "rate" if spark.version < "3.2.1" else "rate-micro-batch", + {'numPartitions': spark.sparkContext.defaultParallelism, + 'rowsPerBatch': spark.sparkContext.defaultParallelism, + 'startTimestamp': "*"} if spark.version >= "3.2.1" + else {'numPartitions': spark.sparkContext.defaultParallelism, + 'rowsPerSecond': spark.sparkContext.defaultParallelism}, None), + + (None, "rate", {'numPartitions': spark.sparkContext.defaultParallelism, + 'rowsPerSecond': spark.sparkContext.defaultParallelism}, "3.0.0"), + + (None, "rate-micro-batch", + {'numPartitions': spark.sparkContext.defaultParallelism, + 'rowsPerBatch': spark.sparkContext.defaultParallelism, + 'startTimestamp': "*"}, "3.2.1"), + + ({'streamingSource': 'rate'}, 'rate', + {'numPartitions': spark.sparkContext.defaultParallelism, + 'streamingSource': 'rate', + 'rowsPerSecond': spark.sparkContext.defaultParallelism}, None), + + ({'streamingSource': 'rate', 'rowsPerSecond': 5000}, 'rate', + {'numPartitions': spark.sparkContext.defaultParallelism, + 'streamingSource': 'rate', + 'rowsPerSecond': 5000}, None), + + ({'streamingSource': 'rate', 'numPartitions': 10}, 'rate', + {'numPartitions': 10, 'rowsPerSecond': 10, 'streamingSource': 'rate'}, None), + + ({'streamingSource': 'rate', 'numPartitions': 10, 'rowsPerSecond': 5000}, 'rate', + {'numPartitions': 10, 'rowsPerSecond': 5000, 'streamingSource': 'rate'}, None), + + ({'streamingSource': 'rate-micro-batch'}, 'rate-micro-batch', + {'streamingSource': 'rate-micro-batch', + 'numPartitions': spark.sparkContext.defaultParallelism, + 'startTimestamp': '*', + 'rowsPerBatch': spark.sparkContext.defaultParallelism}, None), + + ({'streamingSource': 'rate-micro-batch', 'numPartitions':20}, 'rate-micro-batch', + {'streamingSource': 'rate-micro-batch', + 'numPartitions': 20, + 'startTimestamp': '*', + 'rowsPerBatch': 20}, None), + + ({'streamingSource': 'rate-micro-batch', 'numPartitions': 20, 'rowsPerBatch': 4300}, + 'rate-micro-batch', + {'streamingSource': 'rate-micro-batch', + 'numPartitions': 20, + 'startTimestamp': '*', + 'rowsPerBatch': 4300}, None), + ]) + def test_prepare_options(self, options_supplied, source_expected, options_expected, spark_version_override): + testDataSpec = dg.DataGenerator(sparkSession=spark, name="test_data_set1", + rows=self.row_count, + partitions=spark.sparkContext.defaultParallelism, + seedMethod='hash_fieldname') + + streaming_source, new_options = testDataSpec._prepareStreamingOptions(options_supplied, spark_version_override) + print("Options supplied:", options_supplied, "streamingSource:", streaming_source) + + assert streaming_source == source_expected, "unexpected streaming source" + + if streaming_source == "rate-micro-batch": + assert "startTimestamp" in new_options + assert "startTimestamp" in options_expected + if options_expected["startTimestamp"] == "*": + options_expected.pop("startTimestamp") + new_options.pop("startTimestamp") + + print("options expected:", options_expected) + + assert new_options == options_expected, "unexpected options" + + @pytest.fixture + def getBaseDir(self, request): time_now = int(round(time.time() * 1000)) - base_dir = "/tmp/testdatagenerator_{}".format(time_now) - test_dir = os.path.join(base_dir, "data1") - checkpoint_dir = os.path.join(base_dir, "checkpoint1") - print(time_now, test_dir, checkpoint_dir) + base_dir = f"/tmp/testdatagenerator_{request.node.originalname}_{time_now}" + yield base_dir + print("cleaning base dir") + shutil.rmtree(base_dir) - new_data_rows = 0 + @pytest.fixture + def getCheckpoint(self, getBaseDir, request): + checkpoint_dir = os.path.join(getBaseDir, "checkpoint1") + os.makedirs(checkpoint_dir) + + yield checkpoint_dir + print("cleaning checkpoint dir") + + @pytest.fixture + def getDataDir(self, getBaseDir, request): + data_dir = os.path.join(getBaseDir, "data1") + os.makedirs(data_dir) + + yield data_dir + print("cleaning data dir") + + + + def test_fixture1(self, getCheckpoint, getDataDir): + print(getCheckpoint) + print(getDataDir) + + def test_streaming_basic_rate(self, getDataDir, getCheckpoint): + test_dir = getDataDir + checkpoint_dir = getCheckpoint try: - os.makedirs(test_dir) - os.makedirs(checkpoint_dir) - testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, - partitions=4, seedMethod='hash_fieldname') + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", + rows=self.row_count, + partitions=spark.sparkContext.defaultParallelism, + seedMethod='hash_fieldname') + .withIdOutput()) + + dfTestData = testDataSpec.build(withStreaming=True, + options={'rowsPerSecond': self.rows_per_second, + 'ageLimit': 1, + 'streamingSource': 'rate'}) + + (dfTestData.writeStream + .option("checkpointLocation", checkpoint_dir) + .outputMode("append") + .format("parquet") + .start(test_dir) + ) + + start_time = time.time() + time.sleep(self.time_to_run) + + # note stopping the stream may produce exceptions - these can be ignored + recent_progress = [] + for x in spark.streams.active: + recent_progress.append(x.recentProgress) + print(x) + x.stop() + + end_time = time.time() + + # read newly written data + df2 = spark.read.format("parquet").load(test_dir) + + new_data_rows = df2.count() + + print("read {} rows from newly written data".format(new_data_rows)) + finally: + pass + + print("*** Done ***") + + print("elapsed time (seconds)", end_time - start_time) + + # check that we have at least one second of data + assert new_data_rows > self.rows_per_second + + def test_streaming_basic_rate_micro_batch(self, getDataDir, getCheckpoint): + test_dir = getDataDir + checkpoint_dir = getCheckpoint + + try: + + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", + rows=self.row_count, + partitions=spark.sparkContext.defaultParallelism, + seedMethod='hash_fieldname') .withIdOutput() - .withColumn("r", FloatType(), expr="floor(rand() * 350) * (86400 + 3600)", - numColumns=self.column_count) .withColumn("code1", IntegerType(), minValue=100, maxValue=200) .withColumn("code2", IntegerType(), minValue=0, maxValue=10) .withColumn("code3", StringType(), values=['a', 'b', 'c']) - .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) - .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) - ) dfTestData = testDataSpec.build(withStreaming=True, - options={'rowsPerSecond': self.rows_per_second}) + options={'rowsPerBatch': 1000, + 'streamingSource': 'rate-micro-batch', + 'startTimestamp': 0}) - (dfTestData - .writeStream - .format("parquet") - .outputMode("append") - .option("path", test_dir) - .option("checkpointLocation", checkpoint_dir) - .start()) + (dfTestData.writeStream + .option("checkpointLocation", checkpoint_dir) + .outputMode("append") + .format("parquet") + .start(test_dir) + ) + + start_time = time.time() + time.sleep(self.time_to_run) + + # note stopping the stream may produce exceptions - these can be ignored + recent_progress = [] + for x in spark.streams.active: + recent_progress.append(x.recentProgress) + print(x) + x.stop() + + end_time = time.time() + + # read newly written data + df2 = spark.read.format("parquet").load(test_dir) + + new_data_rows = df2.count() + + print("read {} rows from newly written data".format(new_data_rows)) + finally: + pass + + print("*** Done ***") + + print("elapsed time (seconds)", end_time - start_time) + + # check that we have at least one second of data + assert new_data_rows > self.rows_per_second + + + def test_streaming_rate_source(self): + print(spark.version) + test_dir, checkpoint_dir, base_dir = self.getDataAndCheckpoint("test1") + + new_data_rows = 0 + + self.makeDataAndCheckpointDirs(test_dir, checkpoint_dir) + + try: + + testDataSpec = self.getTestDataSpec() + + dfTestData = testDataSpec.build(withStreaming=True, + options={'rowsPerSecond': self.rows_per_second, + 'ageLimit': 1, + 'streamingSource': 'rate'}) + + start_time = time.time() + time.sleep(self.time_to_run) + + # note stopping the stream may produce exceptions - these can be ignored + recent_progress = [] + for x in spark.streams.active: + recent_progress.append(x.recentProgress) + print(x) + x.stop() + + end_time = time.time() + + # read newly written data + df2 = spark.read.format("parquet").load(test_dir) + + new_data_rows = df2.count() + + print("read {} rows from newly written data".format(new_data_rows)) + finally: + shutil.rmtree(base_dir) + + print("*** Done ***") + + print("elapsed time (seconds)", end_time - start_time) + + # check that we have at least one second of data + self.assertGreater(new_data_rows, self.rows_per_second) + + + def test_streaming(self): + print(spark.version) + test_dir, checkpoint_dir, base_dir = self.getDataAndCheckpoint("test1") + + new_data_rows = 0 + + self.makeDataAndCheckpointDirs(test_dir, checkpoint_dir) + + try: + + testDataSpec = self.getTestDataSpec() + + dfTestData = testDataSpec.build(withStreaming=True, + options={'rowsPerSecond': self.rows_per_second, + 'ageLimit': 1}) start_time = time.time() time.sleep(self.time_to_run) From 305a16a810f28e0ad8259c74fc6bd2fde7e15236 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Wed, 15 Feb 2023 18:00:01 -0800 Subject: [PATCH 04/26] wip --- dbldatagen/data_generator.py | 144 ++--------- makefile | 8 - python/dev_require_321.txt | 33 --- tests/test_streaming.py | 458 ++++------------------------------- 4 files changed, 73 insertions(+), 570 deletions(-) delete mode 100644 python/dev_require_321.txt diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index fb78ab1b..e3f63e74 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -17,18 +17,8 @@ from .utils import ensure, topologicalSort, DataGenError, deprecated from . _version import _get_spark_version -START_TIMESTAMP_OPTION = "startTimestamp" -ROWS_PER_SECOND_OPTION = "rowsPerSecond" -AGE_LIMIT_OPTION = "ageLimit" -NUM_PARTITIONS_OPTION = "numPartitions" -ROWS_PER_BATCH_OPTION = "rowsPerBatch" -STREAMING_SOURCE_OPTION = "streamingSource" - _OLD_MIN_OPTION = 'min' _OLD_MAX_OPTION = 'max' -RATE_SOURCE = "rate" -RATE_PER_MICRO_BATCH_SOURCE = "rate-micro-batch" -SPARK_RATE_MICROBATCH_VERSION = "3.2.1" _STREAMING_TIMESTAMP_COLUMN = "_source_timestamp" @@ -873,124 +863,32 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): df1 = df1.withColumnRenamed(SPARK_RANGE_COLUMN, self._seedColumnName) else: - df1 = self._getStreamingBaseDataFrame(startId, options) - - return df1 - - def _getStreamingSource(self, options=None, spark_version=None): - """ get streaming source from options + status = ( + f"Generating streaming data frame with ids from {startId} to {end_id} with {id_partitions} partitions") + self.logger.info(status) + self.executionHistory.append(status) - :param options: dictionary of options - :returns: streaming source if present in options (popping option from options), or default if not present + df1 = (self.sparkSession.readStream + .format("rate")) + if options is not None: + if "rowsPerSecond" not in options: + options['rowsPerSecond'] = 1 + if "numPartitions" not in options: + options['numPartitions'] = id_partitions - Default streaming source is computed based on whether we are running on Spark version 3.2.1 or later + for k, v in options.items(): + df1 = df1.option(k, v) + df1 = (df1.load() + .withColumnRenamed("value", self._seedColumnName) + ) - if using spark version 3.2.1 or later - `rate-micro-batch` is used as source, otherwise `rate` is used as source - """ - streaming_source = None - if options is not None: - if STREAMING_SOURCE_OPTION in options: - streaming_source = options[STREAMING_SOURCE_OPTION] - assert streaming_source in [RATE_SOURCE, RATE_PER_MICRO_BATCH_SOURCE], \ - f"Invalid streaming source - only ['{RATE_SOURCE}', ['{RATE_PER_MICRO_BATCH_SOURCE}'] supported" - - if spark_version is None: - spark_version = self.sparkSession.version - - if streaming_source is None: - # if using Spark 3.2.1, then default should be RATE_PER_MICRO_BATCH_SOURCE - if spark_version >= SPARK_RATE_MICROBATCH_VERSION: - streaming_source = RATE_PER_MICRO_BATCH_SOURCE else: - streaming_source = RATE_SOURCE - - return streaming_source - - def _getCurrentSparkTimestamp(self, asLong=False): - """ get current spark timestamp - - :param asLong: if True, returns current spark timestamp as long, string otherwise - """ - if asLong: - return (self.sparkSession.sql(f"select cast(now() as long) as start_timestamp") - .collect()[0]['start_timestamp']) - else: - return (self.sparkSession.sql(f"select cast(now() as string) as start_timestamp") - .collect()[0]['start_timestamp']) - - def _prepareStreamingOptions(self, options=None, spark_version=None): - default_streaming_partitions = (self.partitions if self.partitions is not None - else self.sparkSession.sparkContext.defaultParallelism) - - streaming_source = self._getStreamingSource(options, spark_version) - - if options is None: - new_options = ({ROWS_PER_SECOND_OPTION: default_streaming_partitions} if streaming_source == RATE_SOURCE - else {ROWS_PER_BATCH_OPTION: default_streaming_partitions}) - else: - new_options = options.copy() - - if NUM_PARTITIONS_OPTION in new_options: - streaming_partitions = new_options[NUM_PARTITIONS_OPTION] - else: - streaming_partitions = default_streaming_partitions - new_options[NUM_PARTITIONS_OPTION] = streaming_partitions - - if streaming_source == RATE_PER_MICRO_BATCH_SOURCE: - if START_TIMESTAMP_OPTION not in new_options: - new_options[START_TIMESTAMP_OPTION] = self._getCurrentSparkTimestamp(asLong=True) - - if ROWS_PER_BATCH_OPTION not in new_options: - # generate one row per partition - new_options[ROWS_PER_BATCH_OPTION] = streaming_partitions - - elif streaming_source == RATE_SOURCE: - if ROWS_PER_SECOND_OPTION not in new_options: - new_options[ROWS_PER_SECOND_OPTION] = streaming_partitions - else: - assert streaming_source in [RATE_SOURCE, RATE_PER_MICRO_BATCH_SOURCE], \ - f"Invalid streaming source - only ['{RATE_SOURCE}', ['{RATE_PER_MICRO_BATCH_SOURCE}'] supported" - - return streaming_source, new_options - - def _getStreamingBaseDataFrame(self, startId=0, options=None): - """Generate base streaming data frame""" - end_id = self._rowCount + startId - - # determine streaming source - streaming_source, options = self._prepareStreamingOptions(options) - partitions = options[NUM_PARTITIONS_OPTION] - - if streaming_source == RATE_SOURCE: - status = f"Generating streaming data with rate source with {partitions} partitions" - else: - status = f"Generating streaming data with rate-micro-batch source with {partitions} partitions" - - self.logger.info(status) - self.executionHistory.append(status) - - age_limit_interval = None - - if STREAMING_SOURCE_OPTION in options: - options.pop(STREAMING_SOURCE_OPTION) - - if AGE_LIMIT_OPTION in options: - age_limit_interval = options.pop("ageLimit") - assert age_limit_interval is not None and float(age_limit_interval) > 0.0, "invalid age limit" - - assert AGE_LIMIT_OPTION not in options - assert STREAMING_SOURCE_OPTION not in options - - df1 = self.sparkSession.readStream.format(streaming_source) - - for k, v in options.items(): - df1 = df1.option(k, v) - - df1 = df1.load().withColumnRenamed("value", ColumnGenerationSpec.SEED_COLUMN) + df1 = (df1.option("rowsPerSecond", 1) + .option("numPartitions", id_partitions) + .load() + .withColumnRenamed("value", self._seedColumnName) + ) - if age_limit_interval is not None: - df1 = df1.where(f"""abs(cast(now() as double) - cast(`timestamp` as double )) - < cast({age_limit_interval} as double)""") return df1 def _computeColumnBuildOrder(self): diff --git a/makefile b/makefile index cb823771..5ec0f761 100644 --- a/makefile +++ b/makefile @@ -29,10 +29,6 @@ create-dev-env: @echo "$(OK_COLOR)=> making conda dev environment$(NO_COLOR)" conda create -n $(ENV_NAME) python=3.8.10 -create-dev-env-321: - @echo "$(OK_COLOR)=> making conda dev environment for Spark 3.2.1$(NO_COLOR)" - conda create -n $(ENV_NAME) python=3.8.10 - create-github-build-env: @echo "$(OK_COLOR)=> making conda dev environment$(NO_COLOR)" conda create -n pip_$(ENV_NAME) python=3.8 @@ -41,10 +37,6 @@ install-dev-dependencies: @echo "$(OK_COLOR)=> installing dev environment requirements$(NO_COLOR)" pip install -r python/dev_require.txt -install-dev-dependencies321: - @echo "$(OK_COLOR)=> installing dev environment requirements for Spark 3.2.1$(NO_COLOR)" - pip install -r python/dev_require_321.txt - clean-dev-env: @echo "$(OK_COLOR)=> Cleaning dev environment$(NO_COLOR)" @echo "Current version: $(CURRENT_VERSION)" diff --git a/python/dev_require_321.txt b/python/dev_require_321.txt deleted file mode 100644 index 56846a62..00000000 --- a/python/dev_require_321.txt +++ /dev/null @@ -1,33 +0,0 @@ -# The following packages are used in building the test data generator framework. -# All packages used are already installed in the Databricks runtime environment for version 6.5 or later -numpy==1.20.1 -pandas==1.2.4 -pickleshare==0.7.5 -py4j==0.10.9.3 -pyarrow==4.0.0 -pyspark==3.2.1 -python-dateutil==2.8.1 -six==1.15.0 - -# The following packages are required for development only -wheel==0.36.2 -setuptools==52.0.0 -bumpversion -pytest -pytest-cov -pytest-timeout -rstcheck -prospector - -# The following packages are only required for building documentation and are not required at runtime -sphinx==5.0.0 -sphinx_rtd_theme -nbsphinx -numpydoc==0.8 -pypandoc -ipython==7.16.3 -recommonmark -sphinx-markdown-builder -rst2pdf==0.98 -Jinja2 < 3.1 - diff --git a/tests/test_streaming.py b/tests/test_streaming.py index dbc19edd..8a9eb42b 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -10,380 +10,18 @@ spark = dg.SparkSingleton.getLocalInstance("streaming tests") -class TestStreaming: +class TestStreaming(): row_count = 100000 column_count = 10 - time_to_run = 8 + time_to_run = 10 rows_per_second = 5000 - def getTestDataSpec(self): - testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", - rows=self.row_count, - partitions=spark.sparkContext.defaultParallelism, - seedMethod='hash_fieldname') - .withIdOutput() - .withColumn("r", FloatType(), expr="floor(rand() * 350) * (86400 + 3600)", - numColumns=self.column_count) - .withColumn("code1", IntegerType(), minValue=100, maxValue=200) - .withColumn("code2", IntegerType(), minValue=0, maxValue=10) - .withColumn("code3", StringType(), values=['a', 'b', 'c']) - .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) - .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) - ) - return testDataSpec - - def test_get_current_spark_timestamp(self): - testDataSpec = dg.DataGenerator(sparkSession=spark, name="test_data_set1", - rows=self.row_count, - partitions=spark.sparkContext.defaultParallelism, - seedMethod='hash_fieldname') - ts = testDataSpec._getCurrentSparkTimestamp(asLong=False) - - assert type(ts) is str - assert ts is not None and len(ts.strip()) > 0 - print(ts) - - def test_get_current_spark_timestamp2(self): - testDataSpec = dg.DataGenerator(sparkSession=spark, name="test_data_set1", - rows=self.row_count, - partitions=spark.sparkContext.defaultParallelism, - seedMethod='hash_fieldname') - ts = testDataSpec._getCurrentSparkTimestamp(asLong=True) - - assert(type(ts) is int) - print(ts) - - def test_get_current_spark_version(self): - assert spark.version > "3.0.0" - assert spark.version <= "6.0.0" - - @pytest.mark.parametrize("options_supplied,expected,spark_version_override", - [(None, "rate" if spark.version < "3.2.1" else "rate-micro-batch", None), - (None, "rate", "3.0.0"), - (None, "rate-micro-batch", "3.2.1"), - ({'streamingSource': 'rate'}, 'rate', None), - ({'streamingSource': 'rate-micro-batch'}, 'rate-micro-batch', None)]) - def test_streaming_source_options(self, options_supplied, expected, spark_version_override): - print("options", options_supplied) - testDataSpec = dg.DataGenerator(sparkSession=spark, name="test_data_set1", - rows=self.row_count, - partitions=spark.sparkContext.defaultParallelism, - seedMethod='hash_fieldname') - - result = testDataSpec._getStreamingSource(options_supplied, spark_version_override) - print("Options:", options_supplied, "retval:", result) - - assert result == expected - - @pytest.mark.parametrize("options_supplied,source_expected,options_expected,spark_version_override", - [(None, "rate" if spark.version < "3.2.1" else "rate-micro-batch", - {'numPartitions': spark.sparkContext.defaultParallelism, - 'rowsPerBatch': spark.sparkContext.defaultParallelism, - 'startTimestamp': "*"} if spark.version >= "3.2.1" - else {'numPartitions': spark.sparkContext.defaultParallelism, - 'rowsPerSecond': spark.sparkContext.defaultParallelism}, None), - - (None, "rate", {'numPartitions': spark.sparkContext.defaultParallelism, - 'rowsPerSecond': spark.sparkContext.defaultParallelism}, "3.0.0"), - - (None, "rate-micro-batch", - {'numPartitions': spark.sparkContext.defaultParallelism, - 'rowsPerBatch': spark.sparkContext.defaultParallelism, - 'startTimestamp': "*"}, "3.2.1"), - - ({'streamingSource': 'rate'}, 'rate', - {'numPartitions': spark.sparkContext.defaultParallelism, - 'streamingSource': 'rate', - 'rowsPerSecond': spark.sparkContext.defaultParallelism}, None), - - ({'streamingSource': 'rate', 'rowsPerSecond': 5000}, 'rate', - {'numPartitions': spark.sparkContext.defaultParallelism, - 'streamingSource': 'rate', - 'rowsPerSecond': 5000}, None), - - ({'streamingSource': 'rate', 'numPartitions': 10}, 'rate', - {'numPartitions': 10, 'rowsPerSecond': 10, 'streamingSource': 'rate'}, None), - - ({'streamingSource': 'rate', 'numPartitions': 10, 'rowsPerSecond': 5000}, 'rate', - {'numPartitions': 10, 'rowsPerSecond': 5000, 'streamingSource': 'rate'}, None), - - ({'streamingSource': 'rate-micro-batch'}, 'rate-micro-batch', - {'streamingSource': 'rate-micro-batch', - 'numPartitions': spark.sparkContext.defaultParallelism, - 'startTimestamp': '*', - 'rowsPerBatch': spark.sparkContext.defaultParallelism}, None), - - ({'streamingSource': 'rate-micro-batch', 'numPartitions':20}, 'rate-micro-batch', - {'streamingSource': 'rate-micro-batch', - 'numPartitions': 20, - 'startTimestamp': '*', - 'rowsPerBatch': 20}, None), - - ({'streamingSource': 'rate-micro-batch', 'numPartitions': 20, 'rowsPerBatch': 4300}, - 'rate-micro-batch', - {'streamingSource': 'rate-micro-batch', - 'numPartitions': 20, - 'startTimestamp': '*', - 'rowsPerBatch': 4300}, None), - ]) - def test_prepare_options(self, options_supplied, source_expected, options_expected, spark_version_override): - testDataSpec = dg.DataGenerator(sparkSession=spark, name="test_data_set1", - rows=self.row_count, - partitions=spark.sparkContext.defaultParallelism, - seedMethod='hash_fieldname') - - streaming_source, new_options = testDataSpec._prepareStreamingOptions(options_supplied, spark_version_override) - print("Options supplied:", options_supplied, "streamingSource:", streaming_source) - - assert streaming_source == source_expected, "unexpected streaming source" - - if streaming_source == "rate-micro-batch": - assert "startTimestamp" in new_options - assert "startTimestamp" in options_expected - if options_expected["startTimestamp"] == "*": - options_expected.pop("startTimestamp") - new_options.pop("startTimestamp") - - print("options expected:", options_expected) - - assert new_options == options_expected, "unexpected options" - - @pytest.fixture - def getBaseDir(self, request): - time_now = int(round(time.time() * 1000)) - base_dir = f"/tmp/testdatagenerator_{request.node.originalname}_{time_now}" - yield base_dir - print("cleaning base dir") - shutil.rmtree(base_dir) - - @pytest.fixture - def getCheckpoint(self, getBaseDir, request): - checkpoint_dir = os.path.join(getBaseDir, "checkpoint1") - os.makedirs(checkpoint_dir) - - yield checkpoint_dir - print("cleaning checkpoint dir") - @pytest.fixture - def getDataDir(self, getBaseDir, request): - data_dir = os.path.join(getBaseDir, "data1") - os.makedirs(data_dir) - - yield data_dir - print("cleaning data dir") - - - - def test_fixture1(self, getCheckpoint, getDataDir): - print(getCheckpoint) - print(getDataDir) - - def test_streaming_basic_rate(self, getDataDir, getCheckpoint): - test_dir = getDataDir - checkpoint_dir = getCheckpoint - - try: - - testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", - rows=self.row_count, - partitions=spark.sparkContext.defaultParallelism, - seedMethod='hash_fieldname') - .withIdOutput()) - - dfTestData = testDataSpec.build(withStreaming=True, - options={'rowsPerSecond': self.rows_per_second, - 'ageLimit': 1, - 'streamingSource': 'rate'}) - - (dfTestData.writeStream - .option("checkpointLocation", checkpoint_dir) - .outputMode("append") - .format("parquet") - .start(test_dir) - ) - - start_time = time.time() - time.sleep(self.time_to_run) - - # note stopping the stream may produce exceptions - these can be ignored - recent_progress = [] - for x in spark.streams.active: - recent_progress.append(x.recentProgress) - print(x) - x.stop() - - end_time = time.time() - - # read newly written data - df2 = spark.read.format("parquet").load(test_dir) - - new_data_rows = df2.count() - - print("read {} rows from newly written data".format(new_data_rows)) - finally: - pass - - print("*** Done ***") - - print("elapsed time (seconds)", end_time - start_time) - - # check that we have at least one second of data - assert new_data_rows > self.rows_per_second - - def test_streaming_basic_rate_micro_batch(self, getDataDir, getCheckpoint): - test_dir = getDataDir - checkpoint_dir = getCheckpoint - - try: - - testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", - rows=self.row_count, - partitions=spark.sparkContext.defaultParallelism, - seedMethod='hash_fieldname') - .withIdOutput() - .withColumn("code1", IntegerType(), minValue=100, maxValue=200) - .withColumn("code2", IntegerType(), minValue=0, maxValue=10) - .withColumn("code3", StringType(), values=['a', 'b', 'c']) - ) - - dfTestData = testDataSpec.build(withStreaming=True, - options={'rowsPerBatch': 1000, - 'streamingSource': 'rate-micro-batch', - 'startTimestamp': 0}) - - (dfTestData.writeStream - .option("checkpointLocation", checkpoint_dir) - .outputMode("append") - .format("parquet") - .start(test_dir) - ) - - start_time = time.time() - time.sleep(self.time_to_run) - - # note stopping the stream may produce exceptions - these can be ignored - recent_progress = [] - for x in spark.streams.active: - recent_progress.append(x.recentProgress) - print(x) - x.stop() - - end_time = time.time() - - # read newly written data - df2 = spark.read.format("parquet").load(test_dir) - - new_data_rows = df2.count() - - print("read {} rows from newly written data".format(new_data_rows)) - finally: - pass - - print("*** Done ***") - - print("elapsed time (seconds)", end_time - start_time) - - # check that we have at least one second of data - assert new_data_rows > self.rows_per_second - - - def test_streaming_rate_source(self): - print(spark.version) - test_dir, checkpoint_dir, base_dir = self.getDataAndCheckpoint("test1") - - new_data_rows = 0 - - self.makeDataAndCheckpointDirs(test_dir, checkpoint_dir) - - try: - - testDataSpec = self.getTestDataSpec() - - dfTestData = testDataSpec.build(withStreaming=True, - options={'rowsPerSecond': self.rows_per_second, - 'ageLimit': 1, - 'streamingSource': 'rate'}) - - start_time = time.time() - time.sleep(self.time_to_run) - - # note stopping the stream may produce exceptions - these can be ignored - recent_progress = [] - for x in spark.streams.active: - recent_progress.append(x.recentProgress) - print(x) - x.stop() - - end_time = time.time() - - # read newly written data - df2 = spark.read.format("parquet").load(test_dir) - - new_data_rows = df2.count() - - print("read {} rows from newly written data".format(new_data_rows)) - finally: - shutil.rmtree(base_dir) - - print("*** Done ***") - - print("elapsed time (seconds)", end_time - start_time) - - # check that we have at least one second of data - self.assertGreater(new_data_rows, self.rows_per_second) - - - def test_streaming(self): - print(spark.version) - test_dir, checkpoint_dir, base_dir = self.getDataAndCheckpoint("test1") - - new_data_rows = 0 - - self.makeDataAndCheckpointDirs(test_dir, checkpoint_dir) - - try: - - testDataSpec = self.getTestDataSpec() - - dfTestData = testDataSpec.build(withStreaming=True, - options={'rowsPerSecond': self.rows_per_second, - 'ageLimit': 1}) - - start_time = time.time() - time.sleep(self.time_to_run) - - # note stopping the stream may produce exceptions - these can be ignored - recent_progress = [] - for x in spark.streams.active: - recent_progress.append(x.recentProgress) - print(x) - x.stop() - - end_time = time.time() - - # read newly written data - df2 = spark.read.format("parquet").load(test_dir) - - new_data_rows = df2.count() - - print("read {} rows from newly written data".format(new_data_rows)) - finally: - shutil.rmtree(base_dir) - - print("*** Done ***") - - print("elapsed time (seconds)", end_time - start_time) - - # check that we have at least one second of data - self.assertGreater(new_data_rows, self.rows_per_second) - - def test_streaming_with_age_limit(self): - print(spark.version) - + def getStreamingDirs(self): time_now = int(round(time.time() * 1000)) - base_dir = "/tmp/testdatagenerator2_{}".format(time_now) - test_dir = os.path.join(base_dir, "data") - + base_dir = "/tmp/testdatagenerator_{}".format(time_now) + print("test dir created") + data_dir = os.path.join(base_dir, "data") checkpoint_dir = os.path.join(base_dir, "checkpoint") os.makedirs(data_dir) os.makedirs(checkpoint_dir) @@ -403,51 +41,61 @@ def test_streaming(self, getStreamingDirs, seedColumnName): partitions=4, seedMethod='hash_fieldname', seedColumnName=seedColumnName)) else: testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, - partitions=4, seedMethod='hash_fieldname') - .withIdOutput() - .withColumn("r", FloatType(), expr="floor(rand() * 350) * (86400 + 3600)", - numColumns=self.column_count) - .withColumn("code1", IntegerType(), minValue=100, maxValue=200) - .withColumn("code2", IntegerType(), minValue=0, maxValue=10) - .withColumn("code3", StringType(), values=['a', 'b', 'c']) - .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) - .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + partitions=4, seedMethod='hash_fieldname')) - ) + testDataSpec = (testDataSpec + .withIdOutput() + .withColumn("r", FloatType(), expr="floor(rand() * 350) * (86400 + 3600)", + numColumns=self.column_count) + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code2", IntegerType(), minValue=0, maxValue=10) + .withColumn("code3", StringType(), values=['a', 'b', 'c']) + .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) - dfTestData = testDataSpec.build(withStreaming=True, - options={'rowsPerSecond': self.rows_per_second, - 'ageLimit': 1}) + ) - (dfTestData - .writeStream - .format("parquet") - .outputMode("append") - .option("path", test_dir) - .option("checkpointLocation", checkpoint_dir) - .start()) + dfTestData = testDataSpec.build(withStreaming=True, + options={'rowsPerSecond': self.rows_per_second}) - start_time = time.time() - time.sleep(self.time_to_run) + # check that seed column is in schema + fields = [c.name for c in dfTestData.schema.fields] - # note stopping the stream may produce exceptions - these can be ignored - recent_progress = [] - for x in spark.streams.active: - recent_progress.append(x.recentProgress) - print(x) - x.stop() + if seedColumnName is not None: + assert seedColumnName in fields + assert "id" not in fields if seedColumnName != "id" else True - end_time = time.time() + sq = (dfTestData + .writeStream + .format("parquet") + .outputMode("append") + .option("path", test_dir) + .option("checkpointLocation", checkpoint_dir) + .start()) - # read newly written data - df2 = spark.read.format("parquet").load(test_dir) + # loop until we get one seconds worth of data + start_time = time.time() + elapsed_time = 0 + rows_retrieved = 0 + time_limit = 10.0 + + while elapsed_time < time_limit and rows_retrieved <= self.rows_per_second: + time.sleep(1) + + elapsed_time = time.time() - start_time + + try: + df2 = spark.read.format("parquet").load(test_dir) + rows_retrieved = df2.count() - new_data_rows = df2.count() + # ignore file or metadata not found issues arising from read before stream has written first batch + except Exception as exc: + print("Exception:", exc) - print("read {} rows from newly written data".format(new_data_rows)) - finally: - shutil.rmtree(base_dir) + if sq.isActive: + sq.stop() + end_time = time.time() print("*** Done ***") print("read {} rows from newly written data".format(rows_retrieved)) @@ -529,6 +177,4 @@ def test_streaming_trigger_once(self, getStreamingDirs, seedColumnName): print("elapsed time (seconds)", end_time - start_time) # check that we have at least one second of data - self.assertGreater(new_data_rows, int(self.rows_per_second / 4)) - - + assert rows_retrieved >= self.rows_per_second From 8f95ca5db536a5e685f3fefe6d9c374eb0a066a7 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Wed, 15 Feb 2023 18:07:06 -0800 Subject: [PATCH 05/26] wip --- dbldatagen/enhanced_event_time.py | 138 ++++++++++++++++++++++++++++++ 1 file changed, 138 insertions(+) create mode 100644 dbldatagen/enhanced_event_time.py diff --git a/dbldatagen/enhanced_event_time.py b/dbldatagen/enhanced_event_time.py new file mode 100644 index 00000000..b2e942be --- /dev/null +++ b/dbldatagen/enhanced_event_time.py @@ -0,0 +1,138 @@ +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +This file defines the `EnhancedEventTime` class + +This defines helper methods for implementing enhanced event time +""" + +import logging + + +from pyspark.sql.functions import col, pandas_udf +from pyspark.sql.functions import lit, concat, rand, round as sql_round, array, expr, when, udf, \ + format_string +from pyspark.sql.types import FloatType, IntegerType, StringType, DoubleType, BooleanType, \ + TimestampType, DataType, DateType + +from .column_spec_options import ColumnSpecOptions +from .datagen_constants import RANDOM_SEED_FIXED, RANDOM_SEED_HASH_FIELD_NAME, RANDOM_SEED_RANDOM +from .daterange import DateRange +from .distributions import Normal, DataDistribution +from .nrange import NRange +from .text_generators import TemplateGenerator +from .utils import ensure, coalesce_values + +import dbldatagen as dg +import dbldatagen.distributions as dist + +HASH_COMPUTE_METHOD = "hash" +VALUES_COMPUTE_METHOD = "values" +RAW_VALUES_COMPUTE_METHOD = "raw_values" +AUTO_COMPUTE_METHOD = "auto" +COMPUTE_METHOD_VALID_VALUES = [HASH_COMPUTE_METHOD, + AUTO_COMPUTE_METHOD, + VALUES_COMPUTE_METHOD, + RAW_VALUES_COMPUTE_METHOD] + + + +class EnhancedEventTimeHelper(object): + + def init(self): + pass + + def withEnhancedEventTime(self, + dataspec, + startEventTime=None, + acceleratedEventTimeInterval="10 minutes", + fractionLateArriving=0.1, + lateTimeInterval="6 hours", + jitter=(-0.25, 0.9999), + eventTimeName=None, + baseColumn="timestamp", + keepIntermediateColumns=False): + """ + Implement enhanced event time + + :param dataspec: dataspec to apply enhanced event time to + :param startEventTime: start timestamp of output event time + :param acceleratedEventTimeInterval: interval for accelerated event time (i.e "10 minutes") + :param fractionLateArriving: fraction of late arriving data. range [0.0, 1.0] + :param lateTimeInterval: interval for late arriving events (i.e "6 hours") + :param jitter: jitter factor to avoid strictly increasing order in events + :param eventTimeName: Column name for generated event time column + :param baseColumn: Base column name used for computations of adjusted event time + :param keepIntermediateColumns: Flag to retain intermediate columns in the output, [ debug / test only] + + This adjusts the dataframe to produce IOT style event time that normally increases over time but has a + configurable fraction of late arriving data. It uses a base timestamp column (called `timestamp` by default) + to compute data. There must be a column of this name in the source data frame and it should have the value of + "now()". + + By default `rate` and `rate-micro-batch` streaming sources have this column but + it can be added to other dataframes - either batch or streaming data frames. + + While the overall intent is to support synthetic IOT style simulated device data in a stream, it can be used + with a batch data frame (as long as an appropriate timestamp column is added and designated as the base data + timestamp column. + """ + + assert startEventTime is not None, "value for `startTime` must be specified" + assert dataspec is not None, "dataspec must be specified" + assert 0.0 <= fractionLateArriving <= 1.0, "fractionLateArriving must be in range [0.0, 1.0]" + assert eventTimeName is not None, "eventTimeName argument must be supplied" + + # determine timestamp for start of generation + start_of_generation = \ + dataspec.sparkSession.sql(f"select cast(now() as string) as start_timestamp").collect()[0][ + 'start_timestamp'] + + omitInterimColumns = not keepIntermediateColumns + + retval = (dataspec + .withColumn("_late_arrival_factor1", "double", minValue=0.0, maxValue=1.0, continuous=True, + random=True, + distribution=dist.Beta(2.0, 5.3), omit=omitInterimColumns) + .withColumn("_late_arrival_factor", "double", expr="least(_late_arrival_factor1 * 1.17, 1.0)", + baseColumn="_late_arrival_factor1", + omit=omitInterimColumns) + .withColumn("_stream_time", "timestamp", expr=f"{baseColumn}", + omit=omitInterimColumns, baseColumn=f"{baseColumn}") + .withColumn("_data_gen_time", "timestamp", expr="now()", omit=omitInterimColumns) + .withColumn("_difference_factor", "double", + expr=f"cast(_stream_time as double) - cast(TIMESTAMP '{start_of_generation}' as double)", + baseColumns=["_stream_time"], + omit=omitInterimColumns) + .withColumn("_jitter_within_event_interval", "double", minValue=jitter[0], maxValue=jitter[1], + continuous=True, random=True, + omit=omitInterimColumns) + .withColumn("_late_arrival_prob", "double", minValue=0.0, maxValue=1.0, continuous=True, + random=True, + omit=omitInterimColumns) + .withColumn("_late_arrival", "boolean", + expr=f"case when _late_arrival_prob <= {fractionLateArriving} then true else false end", + baseColumns=["_late_arrival_prob"], + omit=omitInterimColumns) + .withColumn("_ontime_event_ts", "timestamp", + expr=f"""greatest(TIMESTAMP '{startEventTime}' + + ((_difference_factor + _jitter_within_event_interval) + * INTERVAL {acceleratedEventTimeInterval}), + TIMESTAMP '{startEventTime}') """, + baseColumns=["_difference_factor", "_jitter_within_event_interval"], + omit=omitInterimColumns) + .withColumn("_late_arrival_ts", "timestamp", + expr=f"""greatest(_ontime_event_ts - (_late_arrival_factor * INTERVAL {lateTimeInterval}), + TIMESTAMP '{startEventTime}') + """, + baseColumns=["_ontime_event_ts", "_late_arrival_factor"], + omit=omitInterimColumns) + + # generate event time column + .withColumn(eventTimeName, "timestamp", + expr="case when _late_arrival then _late_arrival_ts else _ontime_event_ts end", + baseColumns=["_late_arrival", "_late_arrival_ts", "_ontime_event_ts"]) + ) + return retval From 13c3a91edd4d15e5b3ece19589ea83d782ab9a11 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Thu, 16 Feb 2023 23:32:15 -0800 Subject: [PATCH 06/26] wip --- CHANGELOG.md | 5 +++++ tests/test_streaming.py | 8 ++++++-- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6f67ab0c..8fb27fb2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -8,6 +8,8 @@ All notable changes to the Databricks Labs Data Generator will be documented in #### Changed * Refactoring of template text generation for better performance via vectorized implementation * Additional migration of tests to use of `pytest` +* Changed parsing of build options for data generator to support use of custom streaming +* Documentation updates in support of new features such as streaming, complex structures etc #### Fixed * added type parsing support for binary and constructs such as `nvarchar(10)` @@ -16,6 +18,9 @@ All notable changes to the Databricks Labs Data Generator will be documented in #### Added * Ability to change name of seed column to custom name (defaults to `id`) * Added type parsing support for structs, maps and arrays and combinations of the above +* Added support for additional streaming source types and for use of custom streaming sources +* Added support for use of file reads as a streaming source (for seed and timestamp columns only) +* Added support for complex event time in streaming scenarios. It may also be used in batch scenarios for testing #### Notes * column definitions for map, struct or array must use `expr` attribute to initialize field. Defaults to `NULL` diff --git a/tests/test_streaming.py b/tests/test_streaming.py index 8a9eb42b..7dff3bc6 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -32,7 +32,9 @@ def getStreamingDirs(self): shutil.rmtree(base_dir, ignore_errors=True) print(f"\n\n*** test dir [{base_dir}] deleted") - @pytest.mark.parametrize("seedColumnName", ["id", "_id", None]) + @pytest.mark.parametrize("seedColumnName", ["id", + "_id", + None]) def test_streaming(self, getStreamingDirs, seedColumnName): base_dir, test_dir, checkpoint_dir = getStreamingDirs @@ -104,7 +106,9 @@ def test_streaming(self, getStreamingDirs, seedColumnName): # check that we have at least one second of data assert rows_retrieved >= self.rows_per_second - @pytest.mark.parametrize("seedColumnName", ["id", "_id", None]) + @pytest.mark.parametrize("seedColumnName", ["id", + "_id", + None]) def test_streaming_trigger_once(self, getStreamingDirs, seedColumnName): base_dir, test_dir, checkpoint_dir = getStreamingDirs From 11cfacd347ff42b25f8ccab926b48e0386cec780 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Mon, 27 Feb 2023 16:51:36 -0800 Subject: [PATCH 07/26] documentation updates --- dbldatagen/data_generator.py | 50 +++++++ docs/source/extending_text_generation.rst | 2 +- docs/source/generating_cdc_data.rst | 2 +- docs/source/generating_json_data.rst | 2 +- docs/source/index.rst | 3 +- docs/source/using_streaming_data.rst | 152 ++++++++++++++++++-- docs/source/working_with_dlt.rst | 160 ++++++++++++++++++++++ tests/test_streaming.py | 75 ++++++++++ 8 files changed, 433 insertions(+), 13 deletions(-) create mode 100644 docs/source/working_with_dlt.rst diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index e3f63e74..b0b326a3 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -20,6 +20,14 @@ _OLD_MIN_OPTION = 'min' _OLD_MAX_OPTION = 'max' +_STREAMING_SOURCE_OPTION = "dbldatagen.streaming.source" +_STREAMING_SCHEMA_OPTION = "dbldatagen.streaming.sourceSchema" +_STREAMING_PATH_OPTION = "dbldatagen.streaming.sourcePath" +_STREAMING_ID_FIELD_OPTION = "dbldatagen.streaming.sourceIdField" +_STREAMING_TIMESTAMP_FIELD_OPTION = "dbldatagen.streaming.sourceTimestampField" +_STREAMING_GEN_TIMESTAMP_OPTION = "dbldatagen.streaming.generateTimestamp" +_BUILD_OPTION_PREFIX = "dbldatagen." + _STREAMING_TIMESTAMP_COLUMN = "_source_timestamp" @@ -976,6 +984,48 @@ def computeBuildPlan(self): self.buildPlanComputed = True return self + def _parseBuildOptions(self, options): + """ Parse build options + + Parse build options into tuple of dictionaries - (datagen options, passthrough options, unsupported options) + + where + + - `datagen options` is dictionary of options to be interpreted by the data generator + - `passthrough options` is dictionary of options to be passed through to the underlying base dataframe + - `supported options` is dictionary of options that are not supported + + :param options: Dict of options to control generating of data + :returns: tuple of options dictionaries - (datagen_options, passthrough_options, unsupported options) + + """ + passthrough_options = {} + unsupported_options = {} + datagen_options = {} + + supported_options = [_STREAMING_SOURCE_OPTION, + _STREAMING_SCHEMA_OPTION, + _STREAMING_PATH_OPTION, + _STREAMING_ID_FIELD_OPTION, + _STREAMING_TIMESTAMP_FIELD_OPTION, + _STREAMING_GEN_TIMESTAMP_OPTION + ] + + if options is not None: + for k, v in options.items(): + if isinstance(k, str): + if k.startswith(_BUILD_OPTION_PREFIX): + if k in supported_options: + datagen_options[k] = v + else: + unsupported_options[k] = v + else: + passthrough_options[k] = v + else: + unsupported_options[k] = v + + return datagen_options, passthrough_options, unsupported_options + def build(self, withTempView=False, withView=False, withStreaming=False, options=None): """ build the test data set from the column definitions and return a dataframe for it diff --git a/docs/source/extending_text_generation.rst b/docs/source/extending_text_generation.rst index 55835e17..f2d091b4 100644 --- a/docs/source/extending_text_generation.rst +++ b/docs/source/extending_text_generation.rst @@ -1,4 +1,4 @@ -.. Test Data Generator documentation master file, created by +.. Databricks Labs Data Generator documentation master file, created by sphinx-quickstart on Sun Jun 21 10:54:30 2020. You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. diff --git a/docs/source/generating_cdc_data.rst b/docs/source/generating_cdc_data.rst index 6528f4df..68cbe265 100644 --- a/docs/source/generating_cdc_data.rst +++ b/docs/source/generating_cdc_data.rst @@ -1,4 +1,4 @@ -.. Test Data Generator documentation master file, created by +.. Databricks Labs Data Generator documentation master file, created by sphinx-quickstart on Sun Jun 21 10:54:30 2020. You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. diff --git a/docs/source/generating_json_data.rst b/docs/source/generating_json_data.rst index a49cd0bb..b2e4564e 100644 --- a/docs/source/generating_json_data.rst +++ b/docs/source/generating_json_data.rst @@ -1,4 +1,4 @@ -.. Test Data Generator documentation master file, created by +.. Databricks Labs Data Generator documentation master file, created by sphinx-quickstart on Sun Jun 21 10:54:30 2020. You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. diff --git a/docs/source/index.rst b/docs/source/index.rst index 4eecc0ac..73f6751c 100644 --- a/docs/source/index.rst +++ b/docs/source/index.rst @@ -31,10 +31,11 @@ As it is installable via `%pip install`, it can also be incorporated in environm Using data distributions Options for column specification Generating repeatable data - Using streaming data + Producing synthetic streaming data Generating JSON and structured column data Generating Change Data Capture (CDC) data Using multiple tables + Working with Delta Live Tables Extending text generation Troubleshooting data generation diff --git a/docs/source/using_streaming_data.rst b/docs/source/using_streaming_data.rst index 6b9f8473..bf2aeafc 100644 --- a/docs/source/using_streaming_data.rst +++ b/docs/source/using_streaming_data.rst @@ -1,27 +1,63 @@ -.. Test Data Generator documentation master file, created by +.. Databricks Labs Data Generator documentation master file, created by sphinx-quickstart on Sun Jun 21 10:54:30 2020. You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. -Using streaming data -==================== +Producing synthetic streaming data +=================================== +The Databricks Labs Data Generator can be used to generate synthetic streaming data using a variety of +streaming sources. + +When generating streaming data, the number of rows to be generated in the original construction of the data spec is +either ignored, and the number of rows and speed at which they are generated depends on the options passed to +the `build` method. + +When generating batch data, the data generation process begins with generating a base dataframe of seed values +using the `spark.range` method. +The rules for additional column generation are then applied to this base data frame. + +For generation of streaming data, the base data frame is generated using one of the `spark.readStream` variations. +This is controlled by passing the argument `withStreaming=True` to the `build` method of the DataGenerator instance. +Additional options may be passed to control rate at which synthetic rows are generated. + +.. note:: + Prior to this release, only the structured streaming `rate` source was available to generate streaming data. But with + this release, you can use a `rate-micro-batch` source, a file source (for streaming reads from data in files such as + delta, parquet or other file formats). Custom formats should also be supported through the new streaming mechanisms + +In theory, any of the structured streaming sources are supported but we do not test compatibility for sources other +than the `rate`, `rate-micro-batch` and file based data sources. + +.. seealso:: + See the following links for more details: + + * `Spark Structured Streaming data sources `_ + +Generating Streaming Data +------------------------- You can make any data spec into a streaming data frame by passing additional options to the ``build`` method. -If the ``withStreaming`` option is used when building the data set, it will use a streaming rate source to generate -the data. You can control the streaming rate with the option ``rowsPerSecond``. +If the ``withStreaming`` option is used when building the data set, it will use a streaming source to generate +the data. By default, this will use a structured streaming `rate` data source as the base data frame. + +When using the `rate` streaming source, you can control the streaming rate with the option ``rowsPerSecond``. -In this case, the row count is ignored. +When using streaming data sources, the row count specified in the call to the DataGenerator instance constructor +is ignored. In most cases, no further changes are needed to run the data generation as a streaming data generator. -As the generated data frame is a normal spark streaming data frame, all the same caveats and features apply. +As the generated data frame is a normal spark streaming data frame, all of the structured streaming caveats +and features apply. Example 1: site code and technology ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ .. code-block:: python + :linenos: + :emphasize-lines: 35 from datetime import timedelta, datetime import math @@ -62,6 +98,71 @@ Example 1: site code and technology display(df) +Customizing streaming data generation +------------------------------------- + +You can customize how the streaming data is generated using the options to the ``build`` command. + +There are two types of options: + + * Options that are interpreted by the streaming data generation process. These options begin with `'dbldatagen.'` + + * Options that are passed through to the underlying streaming data frame. All other options are passed through to the + `options` method of the underlying dataframe. + +.. list-table:: **Data generation options for generating streaming data** + :header-rows: 1 + + * - Option + - Usage + + * - `dbldatagen.streaming.source` + - Type of streaming source to generate. Defaults to `rate` + + * - `dbldatagen.streaming.sourcePath` + - Path for file based data sources. + + * - `dbldatagen.streaming.sourceSchema` + - Schema for source of streaming file sources + + * - `dbldatagen.streaming.sourceIdField` + - Name of source id field - defaults to `value` + + * - `dbldatagen.streaming.sourceTimestampField` + - Name of source timestamp field - defaults to `timestamp` + + * - `dbldatagen.streaming.generateTimestamp` + - if set to `True`, automatically generates a timestamp field if none present + + +The type of the streaming source may be the fully qualified name of a custom streaming source, a built in streaming +source such as `rate` or `rate-micro-batch`, or the name of a file format such as `parquet`, `delta`, or `csv`. + +File based data source support `csv`, `parquet` and `delta` format files or folders of files. Files or folders of +files in `delta` format do not require specification of a schema as it is inferred from the underlying file. + +Files or folders of files in `csv` format require a schema. + +Any options that do not begin with the prefix `dbldatagen.` are passed through to the options method of the underlying +based data frame. + +When a schema is specified for a file based source, the schema should only specify the schema of the fields in the +underlying source, not for additional fields added by the data generation rules. + +.. note:: + Every streaming data source requires a field that can be designated as the seed field or `id` field. + This takes on the same role of the `id` field when batch data generation is used. + + This field will be renamed to the seed field name `id` (or to the custom seed field name, if it + has been overriden in the data generator constructor). + + Many streaming operations also require the designation of a timestamp field to represent event time. This may + be read from the underlying streaming source, or automatically generated. This is also needed if using + enhanced event time (described in a later section). + +What happens if there are other fields in the underlying data source? These are ignored but fields in the generation +spec may refer to them. However, unless a field generation rule replicates the data in the source field, it will not +appear in the generated data. Example 2: IOT style data ^^^^^^^^^^^^^^^^^^^^^^^^^ @@ -70,6 +171,8 @@ The following example shows how to control the length of time to run the streami data generation for. .. code-block:: python + :emphasize-lines: 60,61,62 + :linenos: import time time_to_run = 180 @@ -131,7 +234,8 @@ data generation for. ) - dfTestDataStreaming = testDataSpec.build(withStreaming=True, options={'rowsPerSecond': 500}) + dfTestDataStreaming = testDataSpec.build(withStreaming=True, + options={'rowsPerSecond': 500}) # ... do something with your streaming source here display(dfTestDataStreaming) @@ -150,8 +254,38 @@ terminate the streaming after a specified period of time. except RuntimeError: pass +Using file based streaming sources +---------------------------------- + +Spark structured streaming allows for the use of a dataset storage that has been written in `delta`, +`parquet`, `csv` or other file formats as a streaming source. In particular, the Databricks `delta` format includes +additional enhancements such as transactional isolation that make it suitable as a structured streaming source. + +The Databricks Labs Data Generator allows for the use of files or datasets in `delta`, `parquet` or `csv` format as a +source for the generation of structured streaming data. + +When a file based source is used as a source of structured streaming data, the data generator will use the `seed` or +`id` field from the source and optionally the timestamp field that represents event time. + +When the file contains other fields, these fields may be referred to by other data generation rules (for example in the +body of an `expr` attribute). However, these underlying source fields are not propagated to the output data. + +Simulating late arriving events +------------------------------- + +The data generator also supports simulating the common IOT event time pattern in streaming device data where +most of the events advance in event time as messages are delivered but with a certain percentage of events +arriving late. + +By specifying the use of ehanced event time, you can designate that events increase in event time for most events +but that a certain percentage of the events are delayed up to a configurable time interval. + +This allows for validation of handling of device data with late arriving data, watermarks, stream/stream joins and many +other typical streaming scenarios. + + Using streaming data with Delta tables -^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ +-------------------------------------- If you write the streaming data to a Delta table using a streaming writer, then the Delta table itself can be used as a streaming source diff --git a/docs/source/working_with_dlt.rst b/docs/source/working_with_dlt.rst new file mode 100644 index 00000000..e7057236 --- /dev/null +++ b/docs/source/working_with_dlt.rst @@ -0,0 +1,160 @@ +.. Databricks Labs Data Generator documentation master file, created by + sphinx-quickstart on Sun Jun 21 10:54:30 2020. + You can adapt this file completely to your liking, but it should at least + contain the root `toctree` directive. + +Working with Delta Live Tables +============================== +The Databricks Labs Data Generator can be used within a Delta Live Tables Python pipeline to generate one or more +synthetic data sources that can be used to test or benchmark the Delta Live Tables pipeline. + +To install the library in a Delta Live Tables pipeline, you will need to create a notebook cell as the first cell +of the notebook to install the library via the `%pip` command. + +.. code-block:: python + + %pip install dbldatagen + +Once the Data Generator has been installed, you can use it to create DLT live tables and live streaming tables. + +.. note:: + We recommend using the `rate-micro-batch` streaming source when creating a Delta Live Tables streaming source. + +Creating a batch data source +---------------------------- + +Creating a batch data source in DLT is similar to creating a batch data source in a classic Databricks notebook. + +For example the following code snippet creates a dataset of hypothetical device states at different times as typically +reported by device control boards. + +Typically in industrial controllers, devices report various status codes known as +`tags` at different times. These status codes can include whether the device is available, active or in some form of +error state. So this simulates a set of devices reporting their status at different times as a batch data set. + +.. code-block:: python + + import dbldatagen as dg + import dlt + + sites = ['alpha', 'beta', 'gamma', 'delta', 'phi', 'mu', 'lambda'] + num_sites = len(sites) + num_machines = 20 + device_population = num_sites * num_machines + + data_rows = 20 * 1000000 + partitions_requested = 16 + + machine_states = ['RUNNING', 'IDLE', 'DOWN'] + machine_weights = [10, 5,1 ] + + tags = ['statusCode', 'another notification 1', 'another notification 2', 'another notification 3'] + + starting_datetime="2022-06-01 01:00:00" + end_datetime="2022-09-01 23:59:00" + event_interval = "10 seconds" + + @dlt.table(name="device_data_bronze", partition_cols=["site", "event_date"] ) + def machine_data_raw(): + # create our data generation spec + ds = (dg.DataGenerator(spark, name="mfg_machine_data_set", rows=data_rows, + partitions=partitions_requested, + randomSeedMethod='hash_fieldname') + # use omit = True if you don't want a column to appear in the final output + # but just want to use it as part of generation of another column + .withColumn("internal_site_id", "int", maxValue=num_sites, omit=True, + random=True) + .withColumn("site", "string", values=sites, baseColumn="internal_site_id") + + .withColumn("internal_machine_id", "int", maxValue=num_machines-1, omit=True, + random=True) + .withColumn("machine", "string", prefix="machine", baseColumn="internal_machine_id") + + + # base column entries of machine, site etc means compute these fields first + # and that generated value depends on these fields + .withColumn("machineKey", "string", expr = "concat('/', site, '/', machine)", + baseColumn=["machine", "site"]) + + .withColumn("internal_machine_key", "long", expr = "hash(site, machine)", + omit=True, baseColumn=["machine", "site"]) + + # value is formatted version of base column + .withColumn("deviceId", "string", format="0x%013x", + baseColumn="internal_machine_key") + + .withColumn("tagName", "string", values=tags, random=True) + .withColumn("tagValue", "string", values=machine_states, weights=machine_weights, + random=True) + + + .withColumn("tag_ts", "timestamp", + begin=starting_datetime, + end=end_datetime, + interval=event_interval, + random=True) + + .withColumn("event_date", "date", expr="to_date(tag_ts)", baseColumn="tag_ts") + ) + + # now build and return the data frame + dfTestData = ds.build() + return dfTestData.where("tag_name = 'status_code') + +Creating a streaming data source +-------------------------------- + +describe creating a streaming data source + +Putting it all together +----------------------- +As the synthetic data source defined earlier is a valid DLT table, live table or view, it can be referred to in other +DLT table, live table or view definitions. + +The following snippet uses the `lead` function to determine start and end points of machine states + +.. code-block:: python + + import dlt + + @dlt.table + @dlt.expect("valid_end_timestamp", "< add expectation here>") + def machine_data_silver(): + + import pyspark.sql.functions as F + import pyspark.sql.window as W + + @dlt.table + @dlt.expect("valid_end_timestamp", "end_ts is not null") + @dlt.expect("valid_start_timestamp", "start_ts is not null") + @dlt.expect("site not rolled out", "site <> 'alpha'") + def machine_data_silver2(): + start_pt = spark.conf.get("report.start") + end_pt = spark.conf.get("report.end") + + report_start_ts = f"cast('{start_pt}' as timestamp)" + report_end_ts = f"cast('{end_pt}' as timestamp)" + + # compute start and end periods and supporting data + df_all = dlt.read("machine_data_silver1") + + + # compute start and end timestamps + windowFunction= (W.Window.partitionBy("site","machine","tagName") + .orderBy("site","machine","tagName", "tag_ts")) + + # can use lead / lag in a static dlt table + df_all = (df_all.withColumn("start_ts",F.col("tag_ts")) + .withColumn("_end_ts",F.lead("tag_ts").over(windowFunction)) + .withColumn("end_ts",F.expr(f"""case when _end_ts is null then {report_end_ts} + else _end_ts + end + """)) + .drop("_end_ts") + ) + + # Perform further processing and aggregations + # ,,, + + # return our data frame + return df_all \ No newline at end of file diff --git a/tests/test_streaming.py b/tests/test_streaming.py index 7dff3bc6..c7dabf4b 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -182,3 +182,78 @@ def test_streaming_trigger_once(self, getStreamingDirs, seedColumnName): # check that we have at least one second of data assert rows_retrieved >= self.rows_per_second + + @pytest.mark.parametrize("options,optionsExpected", + [ ({"dbldatagen.streaming.source": "rate"}, + ({"dbldatagen.streaming.source": "rate"}, {}, {})), + ({"dbldatagen.streaming.source": "rate-micro-batch", "rowsPerSecond": 50}, + ({"dbldatagen.streaming.source": "rate-micro-batch"}, {"rowsPerSecond": 50}, {})), + ({"dbldatagen.streaming.source": "rate-micro-batch", + "rowsPerSecond": 50, + "dbldatagen.rows": 100000}, + ({"dbldatagen.streaming.source": "rate-micro-batch"}, + {"rowsPerSecond": 50}, + {"dbldatagen.rows": 100000})) + ]) + def test_option_parsing(self, options, optionsExpected): + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, + partitions=4, seedMethod='hash_fieldname') + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code2", IntegerType(), minValue=0, maxValue=10) + .withColumn("code3", StringType(), values=['a', 'b', 'c']) + .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + ) + + datagen_options, passthrough_options, unsupported_options = testDataSpec._parseBuildOptions(options) + + expected_datagen_options, expected_passthrough_options, expected_unsupported_options = optionsExpected + + assert datagen_options == expected_datagen_options + assert passthrough_options == expected_passthrough_options + assert unsupported_options == expected_unsupported_options + + @pytest.mark.parametrize("options", + [ {"dbldatagen.streaming.source": "parquet", + "dbldatagen.streaming.sourcePath": "/tmp/testStreamingFiles/data1"}, + {"dbldatagen.streaming.source": "csv", + "dbldatagen.streaming.sourcePath": "/tmp/testStreamingFiles/data2"}, + ]) + def test_basic_file_streaming(self, options, getStreamingDirs): + base_dir, test_dir, checkpoint_dir = getStreamingDirs + + # generate file for base of streaming generator + testDataSpecBase = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, + partitions=4, seedMethod='hash_fieldname') + .withColumn('value', "long", expr="id") + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + ) + dfBase = testDataSpecBase.build() + dfBase.write.format(options["dbldatagen.streaming.source"])\ + .mode('overwrite')\ + .save(options["dbldatagen.streaming.source"]) + + # generate streaming data frame + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set2", rows=self.row_count, + partitions=4, seedMethod='hash_fieldname') + .withColumn("a", IntegerType(), minValue=100, maxValue=200) + .withColumn("b", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + ) + dfStreaming = testDataSpecBase.build(withStreaming=True, options=options) + + sq = (dfStreaming + .writeStream + .format("parquet") + .outputMode("append") + .option("path", test_dir) + .option("checkpointLocation", checkpoint_dir) + .start()) + + sq.processAllAvailable() + + dfStreamDataRead = spark.read.format("parquet").load(test_dir) + rows_read = dfStreamDataRead.count() + + assert rows_read == self.row_count + From fa9365bdc8060ec3a5f0f62150cd57836b8c307b Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Mon, 27 Feb 2023 17:04:41 -0800 Subject: [PATCH 08/26] modified base examples for better compatibility with notebooks --- examples/example1.py | 13 ++++++++----- examples/example2.py | 13 +++++++++++++ examples/example3.py | 12 ++++++++++++ examples/example4.py | 10 ++++++++-- examples/example5.py | 9 +++++++-- examples/example6.py | 9 +++++++-- 6 files changed, 55 insertions(+), 11 deletions(-) diff --git a/examples/example1.py b/examples/example1.py index b9722396..262bbcf1 100644 --- a/examples/example1.py +++ b/examples/example1.py @@ -1,3 +1,8 @@ +# Databricks notebook source +# MAGIC %md Example 1 + +# COMMAND ---------- + from datetime import timedelta, datetime from pyspark.sql import SparkSession @@ -20,15 +25,13 @@ ]) # build spark session - -# global spark - +# This is not needed when running examples inside of a Databricks runtime notebook environment spark = SparkSession.builder \ .master("local[4]") \ - .appName("Word Count") \ - .config("spark.some.config.option", "some-value") \ + .appName("Example") \ .getOrCreate() + # will have implied column `id` for ordinal of row x3 = (dg.DataGenerator(sparkSession=spark, name="association_oss_cell_info", rows=100000, partitions=20) .withSchema(schema) diff --git a/examples/example2.py b/examples/example2.py index ef2f37b8..c95f1a13 100644 --- a/examples/example2.py +++ b/examples/example2.py @@ -1,3 +1,8 @@ +# Databricks notebook source +# MAGIC %md Example 2 + +# COMMAND ---------- + from pyspark.sql.functions import col, lit, concat, rand, ceil, floor, round, array, from_unixtime, date_add, datediff, \ unix_timestamp, bround from pyspark.sql.types import LongType, FloatType, IntegerType, StringType, StructType, StructField, DateType, \ @@ -6,6 +11,14 @@ from datetime import timedelta, datetime import math +# build spark session +# This is not needed when running examples inside of a Databricks runtime notebook environment +spark = SparkSession.builder \ + .master("local[4]") \ + .appName("Example") \ + .getOrCreate() + + # examples of generating date time data # def computeIntervals(start, end, interval): diff --git a/examples/example3.py b/examples/example3.py index e392b2ed..57f0ee75 100644 --- a/examples/example3.py +++ b/examples/example3.py @@ -1,6 +1,18 @@ +# Databricks notebook source +# MAGIC %md Example 3 + +# COMMAND ---------- + from datetime import timedelta, datetime import math +# build spark session +# This is not needed when running examples inside of a Databricks runtime notebook environment +spark = SparkSession.builder \ + .master("local[4]") \ + .appName("Example") \ + .getOrCreate() + # examples of use # interval = timedelta(days=1, hours=1) diff --git a/examples/example4.py b/examples/example4.py index f5571faf..e7ef1f93 100644 --- a/examples/example4.py +++ b/examples/example4.py @@ -1,3 +1,8 @@ +# Databricks notebook source +# MAGIC %md Example 4 + +# COMMAND ---------- + from datetime import timedelta, datetime from pyspark.sql import SparkSession @@ -18,10 +23,11 @@ ]) +# build spark session +# This is not needed when running examples inside of a Databricks runtime notebook environment spark = SparkSession.builder \ .master("local[4]") \ - .appName("Word Count") \ - .config("spark.some.config.option", "some-value") \ + .appName("Example") \ .getOrCreate() # will have implied column `id` for ordinal of row diff --git a/examples/example5.py b/examples/example5.py index 0bfc9dc2..3c49565e 100644 --- a/examples/example5.py +++ b/examples/example5.py @@ -1,3 +1,8 @@ +# Databricks notebook source +# MAGIC %md Example 5 + +# COMMAND ---------- + from datetime import timedelta, datetime from pyspark.sql import SparkSession @@ -9,10 +14,10 @@ end = datetime(2018, 10, 1, 6, 0, 0) # build spark session +# This is not needed when running examples inside of a Databricks runtime notebook environment spark = SparkSession.builder \ .master("local[4]") \ - .appName("Word Count") \ - .config("spark.some.config.option", "some-value") \ + .appName("Example") \ .getOrCreate() schema = dg.SchemaParser.parseCreateTable(spark, """ diff --git a/examples/example6.py b/examples/example6.py index 21b849a7..886ad32f 100644 --- a/examples/example6.py +++ b/examples/example6.py @@ -1,3 +1,8 @@ +# Databricks notebook source +# MAGIC %md Example 6 + +# COMMAND ---------- + from datetime import timedelta, datetime from pyspark.sql import SparkSession @@ -9,10 +14,10 @@ end = datetime(2018, 10, 1, 6, 0, 0) # build spark session +# This is not needed when running examples inside of a Databricks runtime notebook environment spark = SparkSession.builder \ .master("local[4]") \ - .appName("Word Count") \ - .config("spark.some.config.option", "some-value") \ + .appName("Example") \ .getOrCreate() schema = dg.SchemaParser.parseCreateTable(spark, """ From 855410cdfd589ba54ef9f365b89dd3b2a5bba9b1 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Thu, 2 Mar 2023 02:57:25 -0800 Subject: [PATCH 09/26] updated changelog --- CHANGELOG.md | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d985909f..9397d61a 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,14 +3,26 @@ ## Change History All notable changes to the Databricks Labs Data Generator will be documented in this file. -### Version 0.3.1 +### Unreleased #### Changed -* Refactoring of template text generation for better performance via vectorized implementation * Additional migration of tests to use of `pytest` * Changed parsing of build options for data generator to support use of custom streaming * Documentation updates in support of new features such as streaming, complex structures etc +#### Fixed + +#### Added +* Added support for additional streaming source types and for use of custom streaming sources +* Added support for use of file reads as a streaming source (for seed and timestamp columns only) +* Added support for complex event time in streaming scenarios. It may also be used in batch scenarios for testing + +### Version 0.3.1 + +#### Changed +* Refactoring of template text generation for better performance via vectorized implementation +* Additional migration of tests to use of `pytest` + #### Fixed * added type parsing support for binary and constructs such as `nvarchar(10)` * Fixed error occurring when schema contains map, array or struct. @@ -18,9 +30,6 @@ All notable changes to the Databricks Labs Data Generator will be documented in #### Added * Ability to change name of seed column to custom name (defaults to `id`) * Added type parsing support for structs, maps and arrays and combinations of the above -* Added support for additional streaming source types and for use of custom streaming sources -* Added support for use of file reads as a streaming source (for seed and timestamp columns only) -* Added support for complex event time in streaming scenarios. It may also be used in batch scenarios for testing #### Notes * column definitions for map, struct or array must use `expr` attribute to initialize field. Defaults to `NULL` From 7a90397a37a92fd374ec75015e189f9a06f01ec9 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Thu, 2 Mar 2023 03:18:47 -0800 Subject: [PATCH 10/26] separated documentation changes to separate branch --- docs/source/extending_text_generation.rst | 2 +- docs/source/generating_cdc_data.rst | 2 +- docs/source/generating_json_data.rst | 2 +- docs/source/working_with_dlt.rst | 131 +--------------------- examples/example1.py | 13 +-- examples/example2.py | 13 --- examples/example3.py | 12 -- examples/example4.py | 10 +- examples/example5.py | 9 +- examples/example6.py | 9 +- 10 files changed, 16 insertions(+), 187 deletions(-) diff --git a/docs/source/extending_text_generation.rst b/docs/source/extending_text_generation.rst index f2d091b4..55835e17 100644 --- a/docs/source/extending_text_generation.rst +++ b/docs/source/extending_text_generation.rst @@ -1,4 +1,4 @@ -.. Databricks Labs Data Generator documentation master file, created by +.. Test Data Generator documentation master file, created by sphinx-quickstart on Sun Jun 21 10:54:30 2020. You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. diff --git a/docs/source/generating_cdc_data.rst b/docs/source/generating_cdc_data.rst index 68cbe265..6528f4df 100644 --- a/docs/source/generating_cdc_data.rst +++ b/docs/source/generating_cdc_data.rst @@ -1,4 +1,4 @@ -.. Databricks Labs Data Generator documentation master file, created by +.. Test Data Generator documentation master file, created by sphinx-quickstart on Sun Jun 21 10:54:30 2020. You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. diff --git a/docs/source/generating_json_data.rst b/docs/source/generating_json_data.rst index b2e4564e..a49cd0bb 100644 --- a/docs/source/generating_json_data.rst +++ b/docs/source/generating_json_data.rst @@ -1,4 +1,4 @@ -.. Databricks Labs Data Generator documentation master file, created by +.. Test Data Generator documentation master file, created by sphinx-quickstart on Sun Jun 21 10:54:30 2020. You can adapt this file completely to your liking, but it should at least contain the root `toctree` directive. diff --git a/docs/source/working_with_dlt.rst b/docs/source/working_with_dlt.rst index e7057236..cdd204bd 100644 --- a/docs/source/working_with_dlt.rst +++ b/docs/source/working_with_dlt.rst @@ -6,7 +6,7 @@ Working with Delta Live Tables ============================== The Databricks Labs Data Generator can be used within a Delta Live Tables Python pipeline to generate one or more -synthetic data sources that can be used to test or benchmark the Delta Live Tables pipeline. +synthetic data sources that can be used to test or benchmark a Delta Live Tables pipeline. To install the library in a Delta Live Tables pipeline, you will need to create a notebook cell as the first cell of the notebook to install the library via the `%pip` command. @@ -25,136 +25,9 @@ Creating a batch data source Creating a batch data source in DLT is similar to creating a batch data source in a classic Databricks notebook. -For example the following code snippet creates a dataset of hypothetical device states at different times as typically -reported by device control boards. - -Typically in industrial controllers, devices report various status codes known as -`tags` at different times. These status codes can include whether the device is available, active or in some form of -error state. So this simulates a set of devices reporting their status at different times as a batch data set. - -.. code-block:: python - - import dbldatagen as dg - import dlt - - sites = ['alpha', 'beta', 'gamma', 'delta', 'phi', 'mu', 'lambda'] - num_sites = len(sites) - num_machines = 20 - device_population = num_sites * num_machines - - data_rows = 20 * 1000000 - partitions_requested = 16 - - machine_states = ['RUNNING', 'IDLE', 'DOWN'] - machine_weights = [10, 5,1 ] - - tags = ['statusCode', 'another notification 1', 'another notification 2', 'another notification 3'] - - starting_datetime="2022-06-01 01:00:00" - end_datetime="2022-09-01 23:59:00" - event_interval = "10 seconds" - - @dlt.table(name="device_data_bronze", partition_cols=["site", "event_date"] ) - def machine_data_raw(): - # create our data generation spec - ds = (dg.DataGenerator(spark, name="mfg_machine_data_set", rows=data_rows, - partitions=partitions_requested, - randomSeedMethod='hash_fieldname') - # use omit = True if you don't want a column to appear in the final output - # but just want to use it as part of generation of another column - .withColumn("internal_site_id", "int", maxValue=num_sites, omit=True, - random=True) - .withColumn("site", "string", values=sites, baseColumn="internal_site_id") - - .withColumn("internal_machine_id", "int", maxValue=num_machines-1, omit=True, - random=True) - .withColumn("machine", "string", prefix="machine", baseColumn="internal_machine_id") - - - # base column entries of machine, site etc means compute these fields first - # and that generated value depends on these fields - .withColumn("machineKey", "string", expr = "concat('/', site, '/', machine)", - baseColumn=["machine", "site"]) - - .withColumn("internal_machine_key", "long", expr = "hash(site, machine)", - omit=True, baseColumn=["machine", "site"]) - - # value is formatted version of base column - .withColumn("deviceId", "string", format="0x%013x", - baseColumn="internal_machine_key") - - .withColumn("tagName", "string", values=tags, random=True) - .withColumn("tagValue", "string", values=machine_states, weights=machine_weights, - random=True) - - - .withColumn("tag_ts", "timestamp", - begin=starting_datetime, - end=end_datetime, - interval=event_interval, - random=True) - - .withColumn("event_date", "date", expr="to_date(tag_ts)", baseColumn="tag_ts") - ) - - # now build and return the data frame - dfTestData = ds.build() - return dfTestData.where("tag_name = 'status_code') Creating a streaming data source -------------------------------- -describe creating a streaming data source - -Putting it all together ------------------------ -As the synthetic data source defined earlier is a valid DLT table, live table or view, it can be referred to in other -DLT table, live table or view definitions. - -The following snippet uses the `lead` function to determine start and end points of machine states - -.. code-block:: python - - import dlt - - @dlt.table - @dlt.expect("valid_end_timestamp", "< add expectation here>") - def machine_data_silver(): - - import pyspark.sql.functions as F - import pyspark.sql.window as W - - @dlt.table - @dlt.expect("valid_end_timestamp", "end_ts is not null") - @dlt.expect("valid_start_timestamp", "start_ts is not null") - @dlt.expect("site not rolled out", "site <> 'alpha'") - def machine_data_silver2(): - start_pt = spark.conf.get("report.start") - end_pt = spark.conf.get("report.end") - - report_start_ts = f"cast('{start_pt}' as timestamp)" - report_end_ts = f"cast('{end_pt}' as timestamp)" - - # compute start and end periods and supporting data - df_all = dlt.read("machine_data_silver1") - - - # compute start and end timestamps - windowFunction= (W.Window.partitionBy("site","machine","tagName") - .orderBy("site","machine","tagName", "tag_ts")) - - # can use lead / lag in a static dlt table - df_all = (df_all.withColumn("start_ts",F.col("tag_ts")) - .withColumn("_end_ts",F.lead("tag_ts").over(windowFunction)) - .withColumn("end_ts",F.expr(f"""case when _end_ts is null then {report_end_ts} - else _end_ts - end - """)) - .drop("_end_ts") - ) - - # Perform further processing and aggregations - # ,,, +You can use the Data Generator to generate a synthetic source for a streaming live table. - # return our data frame - return df_all \ No newline at end of file diff --git a/examples/example1.py b/examples/example1.py index 262bbcf1..b9722396 100644 --- a/examples/example1.py +++ b/examples/example1.py @@ -1,8 +1,3 @@ -# Databricks notebook source -# MAGIC %md Example 1 - -# COMMAND ---------- - from datetime import timedelta, datetime from pyspark.sql import SparkSession @@ -25,13 +20,15 @@ ]) # build spark session -# This is not needed when running examples inside of a Databricks runtime notebook environment + +# global spark + spark = SparkSession.builder \ .master("local[4]") \ - .appName("Example") \ + .appName("Word Count") \ + .config("spark.some.config.option", "some-value") \ .getOrCreate() - # will have implied column `id` for ordinal of row x3 = (dg.DataGenerator(sparkSession=spark, name="association_oss_cell_info", rows=100000, partitions=20) .withSchema(schema) diff --git a/examples/example2.py b/examples/example2.py index c95f1a13..ef2f37b8 100644 --- a/examples/example2.py +++ b/examples/example2.py @@ -1,8 +1,3 @@ -# Databricks notebook source -# MAGIC %md Example 2 - -# COMMAND ---------- - from pyspark.sql.functions import col, lit, concat, rand, ceil, floor, round, array, from_unixtime, date_add, datediff, \ unix_timestamp, bround from pyspark.sql.types import LongType, FloatType, IntegerType, StringType, StructType, StructField, DateType, \ @@ -11,14 +6,6 @@ from datetime import timedelta, datetime import math -# build spark session -# This is not needed when running examples inside of a Databricks runtime notebook environment -spark = SparkSession.builder \ - .master("local[4]") \ - .appName("Example") \ - .getOrCreate() - - # examples of generating date time data # def computeIntervals(start, end, interval): diff --git a/examples/example3.py b/examples/example3.py index 57f0ee75..e392b2ed 100644 --- a/examples/example3.py +++ b/examples/example3.py @@ -1,18 +1,6 @@ -# Databricks notebook source -# MAGIC %md Example 3 - -# COMMAND ---------- - from datetime import timedelta, datetime import math -# build spark session -# This is not needed when running examples inside of a Databricks runtime notebook environment -spark = SparkSession.builder \ - .master("local[4]") \ - .appName("Example") \ - .getOrCreate() - # examples of use # interval = timedelta(days=1, hours=1) diff --git a/examples/example4.py b/examples/example4.py index e7ef1f93..f5571faf 100644 --- a/examples/example4.py +++ b/examples/example4.py @@ -1,8 +1,3 @@ -# Databricks notebook source -# MAGIC %md Example 4 - -# COMMAND ---------- - from datetime import timedelta, datetime from pyspark.sql import SparkSession @@ -23,11 +18,10 @@ ]) -# build spark session -# This is not needed when running examples inside of a Databricks runtime notebook environment spark = SparkSession.builder \ .master("local[4]") \ - .appName("Example") \ + .appName("Word Count") \ + .config("spark.some.config.option", "some-value") \ .getOrCreate() # will have implied column `id` for ordinal of row diff --git a/examples/example5.py b/examples/example5.py index 3c49565e..0bfc9dc2 100644 --- a/examples/example5.py +++ b/examples/example5.py @@ -1,8 +1,3 @@ -# Databricks notebook source -# MAGIC %md Example 5 - -# COMMAND ---------- - from datetime import timedelta, datetime from pyspark.sql import SparkSession @@ -14,10 +9,10 @@ end = datetime(2018, 10, 1, 6, 0, 0) # build spark session -# This is not needed when running examples inside of a Databricks runtime notebook environment spark = SparkSession.builder \ .master("local[4]") \ - .appName("Example") \ + .appName("Word Count") \ + .config("spark.some.config.option", "some-value") \ .getOrCreate() schema = dg.SchemaParser.parseCreateTable(spark, """ diff --git a/examples/example6.py b/examples/example6.py index 886ad32f..21b849a7 100644 --- a/examples/example6.py +++ b/examples/example6.py @@ -1,8 +1,3 @@ -# Databricks notebook source -# MAGIC %md Example 6 - -# COMMAND ---------- - from datetime import timedelta, datetime from pyspark.sql import SparkSession @@ -14,10 +9,10 @@ end = datetime(2018, 10, 1, 6, 0, 0) # build spark session -# This is not needed when running examples inside of a Databricks runtime notebook environment spark = SparkSession.builder \ .master("local[4]") \ - .appName("Example") \ + .appName("Word Count") \ + .config("spark.some.config.option", "some-value") \ .getOrCreate() schema = dg.SchemaParser.parseCreateTable(spark, """ From f62d72879720dbaf66e3f6a9b321a794aebe1944 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Fri, 3 Mar 2023 01:23:20 -0800 Subject: [PATCH 11/26] wip --- dbldatagen/data_generator.py | 63 ++++++++++++++++++++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index b0b326a3..063b8ddc 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -16,6 +16,7 @@ DEFAULT_SEED_COLUMN, SPARK_RANGE_COLUMN, MIN_SPARK_VERSION from .utils import ensure, topologicalSort, DataGenError, deprecated from . _version import _get_spark_version +from .enhanced_event_time import EnhancedEventTimeHelper _OLD_MIN_OPTION = 'min' _OLD_MAX_OPTION = 'max' @@ -52,6 +53,11 @@ class DataGenerator: it is recommended that you use a different name for the seed column - for example `_id`. This may be specified by setting the `seedColumnName` attribute to `_id` + + Note that the number of partitions requested is not guaranteed to be supplied when generating + a streaming data set using a streaming source that generates a different number of partitions. + However for most batch use cases, the requested number of partitions will be generated. + """ # class vars @@ -899,6 +905,63 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): return df1 + def withEnhancedEventTime(self, startEventTime=None, + acceleratedEventTimeInterval="10 minutes", + fractionLateArriving=0.1, + lateTimeInterval="6 hours", + jitter=(-0.25, 0.9999), + eventTimeName="event_ts", + baseColumn="timestamp", + keepIntermediateColumns=False): + """ + Delegate to EnhancedEventTimeHelper to implement enhanced event time + + :param startEventTime: start timestamp of output event time + :param acceleratedEventTimeInterval: interval for accelerated event time (i.e "10 minutes") + :param fractionLateArriving: fraction of late arriving data. range [0.0, 1.0] + :param lateTimeInterval: interval for late arriving events (i.e "6 hours") + :param jitter: jitter factor to avoid strictly increasing order in events + :param eventTimeName: Column name for generated event time column + :param baseColumn: Base column name used for computations of adjusted event time + :param keepIntermediateColumns: Flag to retain intermediate columns in the output, [ debug / test only] + + :returns instance of data generator (note caller object is changed) + + This adjusts the dataframe to produce IOT style event time that normally increases over time but has a + configurable fraction of late arriving data. It uses a base timestamp column (called timestamp) + to compute data. There must be a column of the same name as the base timestamp column in the source data frame + and it should have the value of "now()". + + The modified dataframe will have a new column named using the `eventTimeName` parameter containing the + new timestamp value. + + By default `rate` and `rate-micro-batch` streaming sources have this column but + it can be added to other dataframes - either batch or streaming data frames. + + While the overall intent is to support synthetic IOT style simulated device data in a stream, it can be used + with a batch data frame (as long as an appropriate timestamp column is added and designated as the base data + timestamp column. + """ + + helper = EnhancedEventTimeHelper() + assert baseColumn is not None and len(baseColumn) > 0, "baseColumn argument must be supplied" + + # add metadata for required timestamp field + baseTimestampMeta = RequiredFieldMeta('timestamp', "required by `withEnhancedEventTime` processing") + self._requiredColumns[baseColumn] = baseTimestampMeta + + helper.withEnhancedEventTime(self, + startEventTime, + acceleratedEventTimeInterval, + fractionLateArriving, + lateTimeInterval, + jitter, + eventTimeName, + baseColumn, + keepIntermediateColumns) + + return self + def _computeColumnBuildOrder(self): """ compute the build ordering using a topological sort on dependencies From 1c9e2685931792f3514b9d4f31db49be10a8b12c Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Fri, 3 Mar 2023 01:28:14 -0800 Subject: [PATCH 12/26] wip --- dbldatagen/data_generator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index 063b8ddc..6d5e5e18 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -17,7 +17,7 @@ from .utils import ensure, topologicalSort, DataGenError, deprecated from . _version import _get_spark_version from .enhanced_event_time import EnhancedEventTimeHelper - + _OLD_MIN_OPTION = 'min' _OLD_MAX_OPTION = 'max' From 12fb038e91810e4d81a80d379735a4991f72ba11 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Fri, 3 Mar 2023 10:57:32 -0800 Subject: [PATCH 13/26] wip --- dbldatagen/data_generator.py | 28 ++++++++++++++-- dbldatagen/enhanced_event_time.py | 2 +- dbldatagen/schema_parser.py | 3 +- tests/test_streaming.py | 53 +++++++++++++++++++++++++++++++ 4 files changed, 81 insertions(+), 5 deletions(-) diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index 6d5e5e18..d3e421ce 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -27,10 +27,16 @@ _STREAMING_ID_FIELD_OPTION = "dbldatagen.streaming.sourceIdField" _STREAMING_TIMESTAMP_FIELD_OPTION = "dbldatagen.streaming.sourceTimestampField" _STREAMING_GEN_TIMESTAMP_OPTION = "dbldatagen.streaming.generateTimestamp" +_STREAMING_USE_SOURCE_FIELDS = "dbldatagen.streaming.sourceFields" _BUILD_OPTION_PREFIX = "dbldatagen." _STREAMING_TIMESTAMP_COLUMN = "_source_timestamp" +_STREAMING_SOURCE_RATE = "rate" +_STREAMING_SOURCE_RATE_MICRO_BATCH = "rate-micro-batch" +_STREAMING_SOURCE_NUM_PARTITIONS = "numPartitions" +_STREAMING_SOURCE_ROWS_PER_BATCH = "rowsPerBatch" +_STREAMING_SOURCE_ROWS_PER_SECOND = "rowsPerSecond" class DataGenerator: """ Main Class for test data set generation @@ -159,6 +165,10 @@ def __init__(self, sparkSession=None, name=None, randomSeedMethod=None, self._inferredSchemaFields = [] self.buildPlanComputed = False + # set of columns that must be present during processing + # will be map from column name to reason it is needed + self._requiredColumns = {} + # lets add the seed column self.withColumn(self._seedColumnName, LongType(), nullable=False, implicit=True, omit=True, noWarn=True) self._batchSize = batchSize @@ -860,9 +870,12 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): :returns: Spark data frame for base data that drives the data generation """ + assert self.partitions is not None, "Expecting partition count to be initialized" + id_partitions = self.partitions end_id = self._rowCount + startId - id_partitions = self.partitions if self.partitions is not None else 4 + + build_options, passthrough_options, unsupported_options = self._parseBuildOptions(options) if not streaming: status = (f"Generating data frame with ids from {startId} to {end_id} with {id_partitions} partitions") @@ -877,6 +890,7 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): df1 = df1.withColumnRenamed(SPARK_RANGE_COLUMN, self._seedColumnName) else: + self._applyStreamingDefaults(build_options, passthrough_options) status = ( f"Generating streaming data frame with ids from {startId} to {end_id} with {id_partitions} partitions") self.logger.info(status) @@ -947,8 +961,7 @@ def withEnhancedEventTime(self, startEventTime=None, assert baseColumn is not None and len(baseColumn) > 0, "baseColumn argument must be supplied" # add metadata for required timestamp field - baseTimestampMeta = RequiredFieldMeta('timestamp', "required by `withEnhancedEventTime` processing") - self._requiredColumns[baseColumn] = baseTimestampMeta + self._requiredColumns[baseColumn] = f"Column '{baseColumn}' is required by `withEnhancedEventTime` processing" helper.withEnhancedEventTime(self, startEventTime, @@ -1087,8 +1100,15 @@ def _parseBuildOptions(self, options): else: unsupported_options[k] = v + # add defaults + return datagen_options, passthrough_options, unsupported_options + def _applyStreamingDefaults(self, build_options, passthrough_options): + assert build_options is not None + assert passthrough_options is not None + + def build(self, withTempView=False, withView=False, withStreaming=False, options=None): """ build the test data set from the column definitions and return a dataframe for it @@ -1119,6 +1139,8 @@ def build(self, withTempView=False, withView=False, withStreaming=False, options df1 = self._getBaseDataFrame(self.starting_id, streaming=withStreaming, options=options) + # TODO: check that the required columns are present in the base data frame + self.executionHistory.append("Using Pandas Optimizations {True}") # build columns diff --git a/dbldatagen/enhanced_event_time.py b/dbldatagen/enhanced_event_time.py index b2e942be..bb198c29 100644 --- a/dbldatagen/enhanced_event_time.py +++ b/dbldatagen/enhanced_event_time.py @@ -100,7 +100,7 @@ def withEnhancedEventTime(self, baseColumn="_late_arrival_factor1", omit=omitInterimColumns) .withColumn("_stream_time", "timestamp", expr=f"{baseColumn}", - omit=omitInterimColumns, baseColumn=f"{baseColumn}") + omit=omitInterimColumns, baseColumn=baseColumn) .withColumn("_data_gen_time", "timestamp", expr="now()", omit=omitInterimColumns) .withColumn("_difference_factor", "double", expr=f"cast(_stream_time as double) - cast(TIMESTAMP '{start_of_generation}' as double)", diff --git a/dbldatagen/schema_parser.py b/dbldatagen/schema_parser.py index 389572ca..131d8268 100644 --- a/dbldatagen/schema_parser.py +++ b/dbldatagen/schema_parser.py @@ -132,7 +132,8 @@ def getTypeDefinitionParser(cls): pp.delimitedList(pp.Group(ident + pp.Optional(colon) + pp.Group(type_expr)))) + r_angle # try to capture invalid type name for better error reporting - invalid_type = pp.Word(pp.alphas, pp.alphanums+"_", as_keyword=True) + # use `asKeyword` not `as_keyword` for maximum version compatibility + invalid_type = pp.Word(pp.alphas, pp.alphanums+"_", asKeyword=True) # use left recursion to handle nesting of types type_expr <<= pp.MatchFirst([primitive_type_keyword, array_expr, map_expr, struct_expr, invalid_type]) diff --git a/tests/test_streaming.py b/tests/test_streaming.py index c7dabf4b..6443eb6a 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -248,6 +248,7 @@ def test_basic_file_streaming(self, options, getStreamingDirs): .outputMode("append") .option("path", test_dir) .option("checkpointLocation", checkpoint_dir) + .trigger(once=True) .start()) sq.processAllAvailable() @@ -257,3 +258,55 @@ def test_basic_file_streaming(self, options, getStreamingDirs): assert rows_read == self.row_count + def test_withEventTime_batch(self): + # test it in batch mode + starting_datetime = "2022-06-01 01:00:00" + testDataSpecBase = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, + partitions=4, seedMethod='hash_fieldname') + .withColumn('value', "long", expr="id") + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + .withColumn("timestamp", "timestamp", expr="now()") + .withEnhancedEventTime(startEventTime=starting_datetime, baseColumn="timestamp", + eventTimeName="event_ts") + ) + + df = testDataSpecBase.build() + assert df.count() == self.row_count + + df.show() + + def test_withEventTime_streaming(self, getStreamingDirs): + base_dir, test_dir, checkpoint_dir = getStreamingDirs + + # test it in streaming mode + starting_datetime = "2022-06-01 01:00:00" + testDataSpecBase = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, + partitions=4, seedMethod='hash_fieldname') + .withColumn('value', "long", expr="id") + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + .withColumn("timestamp2", "timestamp", expr="timestamp") + .withEnhancedEventTime(startEventTime=starting_datetime, baseColumn="timestamp2", + eventTimeName="event_ts") + ) + + dfStreaming = testDataSpecBase.build(withStreaming=True) + + sq = (dfStreaming + .writeStream + .format("parquet") + .outputMode("append") + .option("path", test_dir) + .option("checkpointLocation", checkpoint_dir) + .start()) + + sq.awaitTermination(5) + if sq.isActive: + sq.stop() + + dfStreamDataRead = spark.read.format("parquet").load(test_dir) + rows_read = dfStreamDataRead.count() + assert rows_read > 0 + + From b7fafe92feb09b2b24b1239c4190513268853aa6 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sun, 5 Mar 2023 15:25:53 -0800 Subject: [PATCH 14/26] wip --- CHANGELOG.md | 7 + dbldatagen/__init__.py | 2 +- dbldatagen/data_generator.py | 147 +++++++++--- dbldatagen/schema_parser.py | 61 +++++ dbldatagen/utils.py | 55 +++++ docs/source/generating_column_data.rst | 20 +- tests/test_build_planning.py | 298 ++++++++++++++++++++++--- tests/test_schema_parser.py | 38 ++++ tests/test_utils.py | 37 +-- 9 files changed, 583 insertions(+), 82 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 2129bb40..148c9e7e 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -10,6 +10,8 @@ All notable changes to the Databricks Labs Data Generator will be documented in * Changed parsing of build options for data generator to support use of custom streaming * Documentation updates in support of new features such as streaming, complex structures etc * Changed build labelling to comply with PEP440 +* Adjusted column build phase separation (i.e which select statement is used to build columns) so that a + column with a SQL expression can refer to previously created columns without use of a `baseColumn` attribute #### Fixed @@ -17,6 +19,11 @@ All notable changes to the Databricks Labs Data Generator will be documented in * Added support for additional streaming source types and for use of custom streaming sources * Added support for use of file reads as a streaming source (for seed and timestamp columns only) * Added support for complex event time in streaming scenarios. It may also be used in batch scenarios for testing +* Parsing of SQL expressions to determine column dependencies + +#### Notes +* This does not change actual order of column building - but adjusts which phase + ### Version 0.3.1 diff --git a/dbldatagen/__init__.py b/dbldatagen/__init__.py index 631bd750..e0eba2c8 100644 --- a/dbldatagen/__init__.py +++ b/dbldatagen/__init__.py @@ -27,7 +27,7 @@ from .datagen_constants import DEFAULT_RANDOM_SEED, RANDOM_SEED_RANDOM, RANDOM_SEED_FIXED, \ RANDOM_SEED_HASH_FIELD_NAME, MIN_PYTHON_VERSION, MIN_SPARK_VERSION from .utils import ensure, topologicalSort, mkBoundsList, coalesce_values, \ - deprecated, parse_time_interval, DataGenError + deprecated, parse_time_interval, DataGenError, split_list_matching_condition from ._version import __version__ from .column_generation_spec import ColumnGenerationSpec from .column_spec_options import ColumnSpecOptions diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index d3e421ce..ecb89ff9 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -8,22 +8,27 @@ import copy import logging import re +import time +import math from pyspark.sql.types import LongType, IntegerType, StringType, StructType, StructField, DataType from .spark_singleton import SparkSingleton from .column_generation_spec import ColumnGenerationSpec from .datagen_constants import DEFAULT_RANDOM_SEED, RANDOM_SEED_FIXED, RANDOM_SEED_HASH_FIELD_NAME, \ DEFAULT_SEED_COLUMN, SPARK_RANGE_COLUMN, MIN_SPARK_VERSION -from .utils import ensure, topologicalSort, DataGenError, deprecated +from .utils import ensure, topologicalSort, DataGenError, deprecated, split_list_matching_condition from . _version import _get_spark_version from .enhanced_event_time import EnhancedEventTimeHelper - +from .schema_parser import SchemaParser + + _OLD_MIN_OPTION = 'min' _OLD_MAX_OPTION = 'max' _STREAMING_SOURCE_OPTION = "dbldatagen.streaming.source" _STREAMING_SCHEMA_OPTION = "dbldatagen.streaming.sourceSchema" _STREAMING_PATH_OPTION = "dbldatagen.streaming.sourcePath" +_STREAMING_TABLE_OPTION = "dbldatagen.streaming.sourceTable" _STREAMING_ID_FIELD_OPTION = "dbldatagen.streaming.sourceIdField" _STREAMING_TIMESTAMP_FIELD_OPTION = "dbldatagen.streaming.sourceTimestampField" _STREAMING_GEN_TIMESTAMP_OPTION = "dbldatagen.streaming.generateTimestamp" @@ -37,6 +42,13 @@ _STREAMING_SOURCE_NUM_PARTITIONS = "numPartitions" _STREAMING_SOURCE_ROWS_PER_BATCH = "rowsPerBatch" _STREAMING_SOURCE_ROWS_PER_SECOND = "rowsPerSecond" +_STREAM_SOURCE_START_TIMESTAMP = "startTimestamp" + +_STREAMING_SOURCE_TEXT = "text" +_STREAMING_SOURCE_TEXT = "parquet" +_STREAMING_SOURCE_TEXT = "csv" +_STREAMING_SOURCE_TEXT = "json" +_STREAMING_SOURCE_TEXT = "ord" class DataGenerator: """ Main Class for test data set generation @@ -778,7 +790,6 @@ def withColumn(self, colName, colType=StringType(), minValue=None, maxValue=None new_props = {} new_props.update(kwargs) - from .schema_parser import SchemaParser if type(colType) == str: colType = SchemaParser.columnTypeFromString(colType) @@ -892,30 +903,18 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): else: self._applyStreamingDefaults(build_options, passthrough_options) status = ( - f"Generating streaming data frame with ids from {startId} to {end_id} with {id_partitions} partitions") + f"Generating streaming data frame with {id_partitions} partitions") self.logger.info(status) self.executionHistory.append(status) df1 = (self.sparkSession.readStream .format("rate")) - if options is not None: - if "rowsPerSecond" not in options: - options['rowsPerSecond'] = 1 - if "numPartitions" not in options: - options['numPartitions'] = id_partitions - - for k, v in options.items(): - df1 = df1.option(k, v) - df1 = (df1.load() - .withColumnRenamed("value", self._seedColumnName) - ) - else: - df1 = (df1.option("rowsPerSecond", 1) - .option("numPartitions", id_partitions) - .load() - .withColumnRenamed("value", self._seedColumnName) - ) + for k, v in passthrough_options.items(): + df1 = df1.option(k, v) + df1 = (df1.load() + .withColumnRenamed("value", self._seedColumnName) + ) return df1 @@ -992,8 +991,6 @@ def _computeColumnBuildOrder(self): self._seedColumnName, set()) for x in self._allColumnSpecs] - # self.pp_list(dependency_ordering, msg="dependencies") - self.logger.info("dependency list: %s", str(dependency_ordering)) self._buildOrder = list( @@ -1001,15 +998,78 @@ def _computeColumnBuildOrder(self): self.logger.info("columnBuildOrder: %s", str(self._buildOrder)) - # self.pp_list(self._buildOrder, "build order") + self._buildOrder = self._adjustBuildOrderForSqlDependencies(self._buildOrder, self._columnSpecsByName) + return self._buildOrder + def _adjustBuildOrderForSqlDependencies(self, buildOrder, columnSpecsByName): + """ Adjust column build order according to the following heuristics + + 1: if the column being built in a specific build order phase has a SQL expression and it references + other columns in the same build phase (or potentially references them as the expression parsing is + primitive), separate that phase into multiple phases. + + It will also issue a warning if the SQL expression appears to reference a column built later + + :param buildOrder: list of lists of ids - each sublist represents phase of build + :param columnSpecsByName: dictionary to map column names to column specs + :returns: Spark SQL dataframe of generated test data + + """ + new_build_order = [] + + all_columns = set([item for sublist in buildOrder for item in sublist]) + built_columns = [] + prior_phase_built_columns = [] + + # for each phase, evaluate it to see if it needs to be split + for current_phase in buildOrder: + separate_phase_columns = [] + + for columnBeingBuilt in current_phase: + + if columnBeingBuilt in columnSpecsByName: + cs = columnSpecsByName[columnBeingBuilt] + + if cs.expr is not None: + sql_references = SchemaParser.columnsReferencesFromSQLString(cs.expr, filter=all_columns) + + # determine references to columns not yet built + forward_references = set(sql_references) - set(built_columns) + if len(forward_references) > 0: + msg = f"Column '{columnBeingBuilt} may have forward references to {forward_references}." + self.logger.warning(msg) + self.logger.warning("Use `baseColumn` attribute to correct build ordering if necessary") + + references_not_yet_built = set(sql_references) - set(prior_phase_built_columns) + + if len(references_not_yet_built.intersection(set(current_phase))) > 0: + separate_phase_columns.append(columnBeingBuilt) + + # for each column, get the set of sql references and filter against column names + built_columns.append(columnBeingBuilt) + + if len(separate_phase_columns) > 0: + # split phase based on columns in separate_phase_column_list set + revised_phase = split_list_matching_condition(current_phase, lambda el: el in separate_phase_columns) + new_build_order.extend(revised_phase) + else: + # no change to phase + new_build_order.append(current_phase) + + prior_phase_built_columns.extend(current_phase) + + return new_build_order + @property def build_order(self): """ return the build order minus the seed column (which defaults to `id`) The build order will be a list of lists - each list specifying columns that can be built at the same time """ + if not self.buildPlanComputed: + self.computeBuildPlan() + return [x for x in self._buildOrder if x != [self._seedColumnName]] def _getColumnDataTypes(self, columns): @@ -1080,11 +1140,13 @@ def _parseBuildOptions(self, options): datagen_options = {} supported_options = [_STREAMING_SOURCE_OPTION, - _STREAMING_SCHEMA_OPTION, - _STREAMING_PATH_OPTION, - _STREAMING_ID_FIELD_OPTION, - _STREAMING_TIMESTAMP_FIELD_OPTION, - _STREAMING_GEN_TIMESTAMP_OPTION + _STREAMING_SCHEMA_OPTION, + _STREAMING_PATH_OPTION, + _STREAMING_TABLE_OPTION, + _STREAMING_ID_FIELD_OPTION, + _STREAMING_TIMESTAMP_FIELD_OPTION, + _STREAMING_GEN_TIMESTAMP_OPTION, + _STREAMING_USE_SOURCE_FIELDS ] if options is not None: @@ -1108,6 +1170,30 @@ def _applyStreamingDefaults(self, build_options, passthrough_options): assert build_options is not None assert passthrough_options is not None + # default to `rate` streaming source + if _STREAMING_SOURCE_OPTION not in build_options: + build_options[_STREAMING_SOURCE_OPTION] = _STREAMING_SOURCE_RATE + + # setup `numPartitions` if not specified + if build_options[_STREAMING_SOURCE_OPTION] in [_STREAMING_SOURCE_RATE,_STREAMING_SOURCE_RATE_MICRO_BATCH]: + if _STREAMING_SOURCE_NUM_PARTITIONS not in passthrough_options: + passthrough_options[_STREAMING_SOURCE_NUM_PARTITIONS] = self.partitions + + # set up rows per batch if not specified + if build_options[_STREAMING_SOURCE_OPTION] == _STREAMING_SOURCE_RATE: + if _STREAMING_SOURCE_ROWS_PER_SECOND not in passthrough_options: + passthrough_options[_STREAMING_SOURCE_ROWS_PER_SECOND] = 1 + + if build_options[_STREAMING_SOURCE_OPTION] == _STREAMING_SOURCE_RATE_MICRO_BATCH: + if _STREAMING_SOURCE_ROWS_PER_BATCH not in passthrough_options: + passthrough_options[_STREAMING_SOURCE_ROWS_PER_BATCH] = 1 + if _STREAM_SOURCE_START_TIMESTAMP not in passthrough_options: + currentTs = math.floor(time.mktime(time.localtime())) * 1000 + passthrough_options[_STREAM_SOURCE_START_TIMESTAMP] = currentTs + + if build_options[_STREAMING_SOURCE_OPTION] == _STREAMING_SOURCE_TEXT: + self.logger.warning("Use of the `text` format may not work due to lack of type support") + def build(self, withTempView=False, withView=False, withStreaming=False, options=None): """ build the test data set from the column definitions and return a dataframe for it @@ -1168,6 +1254,9 @@ def _buildColumnExpressionsWithSelects(self, df1): Build column generation expressions with selects :param df1: dataframe for base data generator :return: new dataframe + + The data generator build plan is separated into `rounds` of expressions. Each round consists of + expressions that are generated using a single `select` operation """ self.executionHistory.append("Generating data with selects") # generation with selects may be more efficient as less intermediate data frames diff --git a/dbldatagen/schema_parser.py b/dbldatagen/schema_parser.py index 131d8268..913d441c 100644 --- a/dbldatagen/schema_parser.py +++ b/dbldatagen/schema_parser.py @@ -261,6 +261,67 @@ def columnTypeFromString(cls, type_string): return type_construct + @classmethod + def _cleanseSQL(cls, sql_string): + """ Cleanse sql string removing strings so that they are not considered as part of potential column + references + :param sql_string: String representation of SQL expression + :returns: cleansed string + + Any strings identified are replaced with `' '` + """ + assert sql_string is not None, "`sql_string` must be specified" + + # skip over quoted identifiers even if they contain quotes + quoted_ident = pp.QuotedString(quoteChar="`", escQuote="``") + + stringForm1 = pp.Literal('r') + pp.QuotedString(quoteChar="'") + stringForm2 = pp.Literal('r') + pp.QuotedString(quoteChar='"') + stringForm3 = pp.QuotedString(quoteChar="'", escQuote=r"\'") + stringForm4 = pp.QuotedString(quoteChar='"', escQuote=r'\"') + stringForm = stringForm1 ^ stringForm2 ^ stringForm3 ^ stringForm4 + stringForm.set_parse_action(lambda s, loc, toks: "' '") + + parser = quoted_ident ^ stringForm + + transformed_string = parser.transform_string(sql_string) + + return transformed_string + + @classmethod + def columnsReferencesFromSQLString(cls, sql_string, filter=None): + """ Generate a list of possible column references from a SQL string + + This method finds all condidate references to SQL columnn ids in the string + + To avoid the overhead of a full SQL parser, the implementation will simply look for possible field names + + Further improvements may eliminate some common syntax but in current form, reserved words will + also be returned as possible column references. + + So any uses of this must not assume that all possible references are valid column references + + :param sql_string: String representation of SQL expression + :returns: list of possible column references + """ + assert sql_string is not None, "`sql_string` must be specified" + assert filter is None or isinstance(filter, list) or isinstance(filter, set) + + cleansed_sql_string = cls._cleanseSQL(sql_string) + + ident = pp.Word(pp.alphas, pp.alphanums + "_") | pp.QuotedString(quoteChar="`", escQuote="``") + parser = ident + + references = parser.search_string(cleansed_sql_string) + + results = set([item for sublist in references for item in sublist]) + + if filter is not None: + filtered_results = results.intersection(set(filter)) + return list(filtered_results) + else: + return list(results) + @classmethod def parseCreateTable(cls, sparkSession, source_schema): """ Parse a schema from a schema string diff --git a/dbldatagen/utils.py b/dbldatagen/utils.py index b5ca2776..39f3881d 100644 --- a/dbldatagen/utils.py +++ b/dbldatagen/utils.py @@ -75,6 +75,7 @@ def ensure(cond, msg="condition does not hold true"): :raises: `DataGenError` exception if condition does not hold true :returns: Does not return anything but raises exception if condition does not hold """ + def strip_margin(text): return re.sub(r'\n[ \t]*\|', '\n', text) @@ -214,3 +215,57 @@ def parse_time_interval(spec): ) return delta + + +def split_list_matching_condition(lst, cond): + """ Split a list on elements that match a condition + + This will find all matches of a specific condition in the list and split the list into sublists around the + element that matches this condition. + + It will handle multiple matches performing splits on each match. + + For example, the following code will produce the results below: + + x = ['id', 'city_name', 'id', 'city_id', 'city_pop', 'id', 'city_id', 'city_pop','city_id', 'city_pop','id'] + splitListOnCondition(x, lambda el: el == 'id') + + + result: + `[['id'], ['city_name'], ['id'], ['city_id', 'city_pop'], + ['id'], ['city_id', 'city_pop', 'city_id', 'city_pop'], ['id']]` + + :arg lst: list of items to perform condition matches against + :arg cond: lambda function or function taking single argument and returning True or Fals + :returns: list of sublists + """ + + def match_condition(matchList, matchFn): + """Return first index of element of list matching condition""" + if matchList is None or len(matchList) == 0: + return -1 + + for i in range(len(matchList)): + if matchFn(matchList[i]): + return i + + return -1 + + # main code + retval = [] + + if lst is None: + retval = lst + elif len(lst) == 1: + retval = [lst] + else: + ix = match_condition(lst, cond) + if ix != -1: + retval.extend(split_list_matching_condition(lst[0:ix], cond)) + retval.append(lst[ix:ix + 1]) + retval.extend(split_list_matching_condition(lst[ix + 1:], cond)) + else: + retval = [lst] + + # filter out empty lists + return [el for el in retval if el != []] diff --git a/docs/source/generating_column_data.rst b/docs/source/generating_column_data.rst index bd803fa1..e03cd1d0 100644 --- a/docs/source/generating_column_data.rst +++ b/docs/source/generating_column_data.rst @@ -107,13 +107,29 @@ This performs the following actions: - The final set of output fields will be selected (omitting any columns where the ``omit`` attribute was set to **True**) +.. note:: + + Normally the columns will be built in the order specified in the spec. + Use of the `baseColumn` attribute may change the column build ordering. + + This has several implications: -- If a column is referred to in an expression, the ``baseColumn`` attribute must be defined with a dependency +- If a column is referred to in an expression, the ``baseColumn`` attribute may need to be defined with a dependency on that column - If a column uses a base column with a restricted range of values then it is possible that the column will not generate the full range of values in the column generation spec - If the base column is of type ``boolean`` or some other restricted range type, computations on that base value may not produce the expected range of values -- If base column is not specified, you may see errors reporting that the column in an expression does not exist +- If base column is not specified, you may see errors reporting that the column in an expression does not exist. T + This may be fixed by specifying a column dependency using the `baseColumn` attribute + +.. note:: + + The implementation performs primitive scanning of SQL expressions (specified using the `expr` attribute) + to determine if the sql expression depends on + earlier columns and if so, will put the building of the column in a separate phase. + However it does not reorder the building sequence if there is a reference to a column that will be built later in the + SQL expression. + To enforce the dependency, you must use the `baseColumn` attribute to indicate the dependency. diff --git a/tests/test_build_planning.py b/tests/test_build_planning.py index 069b5695..f60a8caf 100644 --- a/tests/test_build_planning.py +++ b/tests/test_build_planning.py @@ -1,10 +1,13 @@ -import unittest +import pytest +import logging from pyspark.sql.types import BooleanType, DateType -from pyspark.sql.types import StructType, StructField, IntegerType, StringType, TimestampType, DecimalType +from pyspark.sql.types import StructType, StructField, IntegerType, StringType, TimestampType, DecimalType, \ + LongType, DoubleType import dbldatagen as dg + schema = StructType([ StructField("PK1", StringType(), True), StructField("XYYZ_IDS", StringType(), True), @@ -141,26 +144,26 @@ StructField("isDeleted", BooleanType(), True) ]) -print("schema", schema) - spark = dg.SparkSingleton.getLocalInstance("unit tests") +@pytest.fixture(scope="class") +def setupLogging(): + FORMAT = '%(asctime)-15s %(message)s' + logging.basicConfig(format=FORMAT) + # Test manipulation and generation of test data for a large schema -class TestBuildPlanning(unittest.TestCase): +class TestBuildPlanning: testDataSpec = None dfTestData = None row_count = 100000 - def setUp(self): - print("setting up") - - @classmethod - def setUpClass(cls): + @pytest.fixture(scope="class") + def sampleDataSpec(self): sale_values = ['RETAIL', 'ONLINE', 'WHOLESALE', 'RETURN'] sale_weights = [1, 5, 5, 1] - cls.testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=cls.row_count, + testDataspec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, partitions=4) .withSchema(schema) .withIdOutput() @@ -181,47 +184,268 @@ def setUpClass(cls): ) + return testDataspec + + + @pytest.fixture() + def sampleDataSet(self, sampleDataSpec): print("Test generation plan") print("=============================") - cls.testDataSpec.explain() + sampleDataSpec.explain() - print("=============================") - print("") - cls.dfTestData = cls.testDataSpec.build() + df = sampleDataSpec.build() + + return df + + def setup_log_capture(self, caplog_object): + """ set up log capture fixture + + Sets up log capture fixture to only capture messages after setup and only + capture warnings and errors + + """ + caplog_object.set_level(logging.WARNING) + + # clear messages from setup + caplog_object.clear() + + def get_log_capture_warngings_and_errors(self, caplog_object, textFlag): + """ + gets count of errors containing specified text + + :param caplog_object: log capture object from fixture + :param textFlag: text to search for to include error or warning in count + :return: count of errors containg text specified in `textFlag` + """ + seed_column_warnings_and_errors = 0 + for r in caplog_object.records: + if (r.levelname == "WARNING" or r.levelname == "ERROR") and textFlag in r.message: + seed_column_warnings_and_errors += 1 - def test_fieldnames_for_schema(self): + return seed_column_warnings_and_errors + + def test_fieldnames_for_schema(self, sampleDataSpec): """Test field names in data spec correspond with schema""" - fieldsFromGenerator = set(self.testDataSpec.getOutputColumnNames()) + fieldsFromGenerator = set(sampleDataSpec.getOutputColumnNames()) fieldsFromSchema = set([fld.name for fld in schema.fields]) # output fields should be same + 'id' field - self.assertEqual(fieldsFromGenerator - fieldsFromSchema, set(['id'])) + assert fieldsFromGenerator - fieldsFromSchema == set(['id']) - def test_explain(self): - self.testDataSpec.computeBuildPlan() - explain_results = self.testDataSpec.explain() - self.assertIsNotNone(explain_results) + def test_explain(self, sampleDataSpec): + sampleDataSpec.computeBuildPlan() + explain_results = sampleDataSpec.explain() + assert explain_results is not None - def test_explain_on_clone(self): - testDataSpec2 = self.testDataSpec.clone() + def test_explain_on_clone(self, sampleDataSpec): + testDataSpec2 = sampleDataSpec.clone() testDataSpec2.computeBuildPlan() explain_results = testDataSpec2.explain(suppressOutput=True) - self.assertIsNotNone(explain_results) + assert explain_results is not None + + def test_build_ordering_basic(self, sampleDataSpec): + build_order = sampleDataSpec.build_order + + # make sure that build order is list of lists + assert build_order is not None + assert isinstance(build_order, list) + + for el in build_order: + assert isinstance(el, list) + + def builtBefore(self, field1, field2, build_order): + """ check if field1 is built before field2""" + + fieldsBuilt = [] + + for phase in build_order: + for el in phase: + if el == field1: + return field2 in fieldsBuilt + fieldsBuilt.append(el) + + return False + + def builtInSeparatePhase(self, field1, field2, build_order): + """ check if field1 is built in separate phase to field2""" + + fieldsBuilt = [] + + for phase in build_order: + for el in phase: + if el == field1: + return field2 not in phase + fieldsBuilt.append(el) + + return False + + + def test_build_ordering_explicit_dependency(self): + gen1 = dg.DataGenerator(sparkSession=spark, name="nested_schema", rows=1000, partitions=4, + seedColumnName="_id") \ + .withColumn("id", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "string", template=r"\w", random=True, omit=True) \ + .withColumn("city_id", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city_pop", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city", "struct", + expr="named_struct('name', city_name, 'id', city_id, 'population', city_pop)", + baseColumns=["city2"]) \ + .withColumn("city2", "struct", + expr="named_struct('name', city_name, 'id', city_id, 'population', city_pop)", + baseColumns=["city_pop"]) \ + .withColumn("city_id2", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True, + baseColumn="city_id") + + build_order = gen1.build_order + + assert self.builtBefore("city_id", "city_name", build_order) + assert self.builtBefore("city", "city2", build_order) + assert self.builtBefore("city2", "city_pop", build_order) + assert self.builtBefore("city_id2", "city_id", build_order) + + assert self.builtBefore("city", "city_name", build_order) + assert self.builtBefore("city", "city_id", build_order) + assert self.builtBefore("city", "city_pop", build_order) + + assert self.builtInSeparatePhase("city", "city_name", build_order) + assert self.builtInSeparatePhase("city", "city_id", build_order) + assert self.builtInSeparatePhase("city", "city_pop", build_order) + + print(gen1.build_order) + + def test_build_ordering_explicit_dependency2(self): + gen1 = dg.DataGenerator(sparkSession=spark, name="nested_schema", rows=1000, partitions=4, + seedColumnName="_id") \ + .withColumn("id", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "string", template=r"\w", random=True, omit=True) \ + .withColumn("city_id", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city_pop", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city", "struct", + expr="named_struct('name', city_name, 'id', city_id, 'population', city_pop)", + baseColumns=["city_name", "city_id", "city_pop"]) \ + .withColumn("city2", "struct", + expr="city", + baseColumns=["city"]) \ + .withColumn("city_id2", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True, + baseColumn="city_id") + + build_order = gen1.build_order + + assert self.builtBefore("city", "city_name", build_order) + assert self.builtBefore("city", "city_id", build_order) + assert self.builtBefore("city", "city_pop", build_order) + assert self.builtInSeparatePhase("city", "city_name", build_order) + assert self.builtInSeparatePhase("city", "city_id", build_order) + assert self.builtInSeparatePhase("city", "city_pop", build_order) + + print(gen1.build_order) + + + def test_build_ordering_implicit_dependency(self): + gen1 = dg.DataGenerator(sparkSession=spark, name="nested_schema", rows=1000, partitions=4, + seedColumnName="_id") \ + .withColumn("id", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "string", template=r"\w", random=True, omit=True) \ + .withColumn("city_id", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city_pop", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city", "struct", + expr="named_struct('name', city_name, 'id', city_id, 'population', city_pop)") + + build_order = gen1.build_order + print(gen1.build_order) + + assert self.builtBefore("city", "city_name", build_order) + assert self.builtBefore("city", "city_id", build_order) + assert self.builtBefore("city", "city_pop", build_order) + assert self.builtInSeparatePhase("city", "city_name", build_order), "fields should be built in separate phase" + assert self.builtInSeparatePhase("city", "city_id", build_order), "fields should be built in separate phase" + assert self.builtInSeparatePhase("city", "city_pop", build_order), "fields should be built in separate phase" + + + def test_expr_attribute(self): + sql_expr = "named_struct('name', city_name, 'id', city_id, 'population', city_pop)" + gen1 = dg.DataGenerator(sparkSession=spark, name="nested_schema", rows=1000, partitions=4, + seedColumnName="_id") \ + .withColumn("id", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "string", template=r"\w", random=True, omit=True) \ + .withColumn("city_id", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city_pop", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city", "struct", + expr=sql_expr) + + columnSpec = gen1.getColumnSpec("city") + + assert columnSpec.expr == sql_expr + + def test_build_ordering_duplicate_names1(self): + gen1 = dg.DataGenerator(sparkSession=spark, name="nested_schema", rows=1000, partitions=4, + seedColumnName="_id") \ + .withColumn("id", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "string", template=r"\w", random=True, omit=True) \ + .withColumn("city_id", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city_pop", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city", "struct", + expr="named_struct('name', city_name, 'id', city_id, 'population', city_pop)") + + build_order = gen1.build_order + print(gen1.build_order) + + df = gen1.build() + + df.show() + + #assert self.builtBefore("city", "city_name", build_order) + #assert self.builtBefore("city", "city_id", build_order) + #assert self.builtBefore("city", "city_pop", build_order) + #assert self.builtInSeparatePhase("city", "city_name", build_order), "fields should be built in separate phase" + #assert self.builtInSeparatePhase("city", "city_id", build_order), "fields should be built in separate phase" + #assert self.builtInSeparatePhase("city", "city_pop", build_order), "fields should be built in separate phase" + + def test_build_ordering_forward_ref(self, caplog): + # caplog fixture captures log content + self.setup_log_capture(caplog) + + gen1 = dg.DataGenerator(sparkSession=spark, name="nested_schema", rows=1000, partitions=4, + seedColumnName="_id") \ + .withColumn("id", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_pop", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city", "struct", + expr="named_struct('name', city_name, 'id', city_id, 'population', city_pop)") \ + .withColumn("city_id", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) + + build_order = gen1.build_order + print(gen1.build_order) + + seed_column_warnings_and_errors = self.get_log_capture_warngings_and_errors(caplog, "forward references") + assert seed_column_warnings_and_errors >= 1, "Should not have error messages about forward references" + + + def test_build_ordering_duplicate_names2(self): + gen1 = dg.DataGenerator(sparkSession=spark, name="nested_schema", rows=1000, partitions=4, + seedColumnName="_id") \ + .withColumn("id", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "long", minValue=1000000, uniqueValues=10000, random=True) \ + .withColumn("city_name", "string", template=r"\w", random=True, omit=True) \ + .withColumn("city_id", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city_pop", "long", minValue=1000000, uniqueValues=10000, random=True, omit=True) \ + .withColumn("city", "struct", + expr="named_struct('name', city_name, 'id', city_id, 'population', city_pop)", + baseColumns=["city_name", "city_id", "city_pop"]) -# run the tests -# if __name__ == '__main__': -# print("Trying to run tests") -# unittest.main(argv=['first-arg-is-ignored'],verbosity=2,exit=False) + build_order = gen1.build_order + print(gen1.build_order) -# def runTests(suites): -# suite = unittest.TestSuite() -# result = unittest.TestResult() -# for testSuite in suites: -# suite.addTest(unittest.makeSuite(testSuite)) -# runner = unittest.TextTestRunner() -# print(runner.run(suite)) + df = gen1.build() + df.show() -# runTests([TestBasicOperation]) + #assert self.builtBefore("city", "city_name", build_order) + #assert self.builtBefore("city", "city_id", build_order) + #assert self.builtBefore("city", "city_pop", build_order) + #assert self.builtInSeparatePhase("city", "city_name", build_order), "fields should be built in separate phase" + #assert self.builtInSeparatePhase("city", "city_id", build_order), "fields should be built in separate phase" + #assert self.builtInSeparatePhase("city", "city_pop", build_order), "fields should be built in separate phase" diff --git a/tests/test_schema_parser.py b/tests/test_schema_parser.py index 74e03574..b529f2ad 100644 --- a/tests/test_schema_parser.py +++ b/tests/test_schema_parser.py @@ -118,3 +118,41 @@ def test_table_definition_parser(self, setupLogging): assert "id" in schema3.fieldNames() assert "name" in schema3.fieldNames() assert "age" in schema3.fieldNames() + + @pytest.mark.parametrize("sqlExpr, expectedText", + [("named_struct('name', city_name, 'id', city_id, 'population', city_pop)", + "named_struct(' ', city_name, ' ', city_id, ' ', city_pop)"), + ("cast(10 as decimal(10)", + "cast(10 as decimal(10)"), + (" ", " "), + ("", ""), + ]) + def test_sql_expression_cleanser(self, sqlExpr, expectedText): + newSql = dg.SchemaParser._cleanseSQL(sqlExpr) + assert sqlExpr == expectedText or sqlExpr != newSql + + assert newSql == expectedText + + @pytest.mark.parametrize("sqlExpr, expectedReferences, filterColumns", + [("named_struct('name', city_name, 'id', city_id, 'population', city_pop)", + ['named_struct', 'city_name', 'city_id', 'city_pop'], + None), + ("named_struct('name', city_name, 'id', city_id, 'population', city_pop)", + [ 'city_name', 'city_pop'], + ['city_name', 'city_pop']), + ("cast(10 as decimal(10)", ['cast', 'as', 'decimal'], None), + ("cast(x as decimal(10)", ['x'], ['x']), + (" ", [], None), + ("", [], None), + ]) + def test_sql_expression_parser(self, sqlExpr, expectedReferences, filterColumns): + references = dg.SchemaParser.columnsReferencesFromSQLString(sqlExpr, filter=filterColumns) + assert references is not None + + assert isinstance(references, list), "expected list of potential column references to be returned" + + print(references) + + assert set(references) == set(expectedReferences) + + diff --git a/tests/test_utils.py b/tests/test_utils.py index db35ac58..24ce35c5 100644 --- a/tests/test_utils.py +++ b/tests/test_utils.py @@ -4,7 +4,7 @@ import pytest from dbldatagen import ensure, mkBoundsList, coalesce_values, deprecated, SparkSingleton, \ - parse_time_interval, DataGenError + parse_time_interval, DataGenError, split_list_matching_condition spark = SparkSingleton.getLocalInstance("unit tests") @@ -75,20 +75,31 @@ def testDatagenExceptionObject(self): assert type(str(testException)) is str self.logger.info(str(testException)) + @pytest.mark.parametrize("lstData,matchFn, expectedData", + [ + (['id', 'city_name', 'id', 'city_id', 'city_pop', 'id', 'city_id', + 'city_pop','city_id', 'city_pop','id'], + lambda el: el == 'id', + [['id'], ['city_name'], ['id'], ['city_id', 'city_pop'], ['id'], + ['city_id', 'city_pop', 'city_id', 'city_pop'], ['id']] + ), + (['id', 'city_name', 'id', 'city_id', 'city_pop', 'id', 'city_id', + 'city_pop2', 'city_id', 'city_pop', 'id'], + lambda el: el in ['id', 'city_pop'], + [['id'], ['city_name'], ['id'], ['city_id'], ['city_pop'], ['id'], + ['city_id', 'city_pop2', 'city_id'], ['city_pop'], ['id']] + ), + ([], lambda el: el == 'id', []), + (['id'], lambda el: el == 'id', [ ['id'] ]), + (['id', 'id'], lambda el: el == 'id', [['id'], ['id']]), + (['no', 'matches'], lambda el: el == 'id', [['no', 'matches']]) + ]) + def testSplitListOnCondition(self, lstData, matchFn, expectedData): + results = split_list_matching_condition(lstData, matchFn) + print(results) -# run the tests -# if __name__ == '__main__': -# print("Trying to run tests") -# unittest.main(argv=['first-arg-is-ignored'],verbosity=2,exit=False) + assert results == expectedData -# def runTests(suites): -# suite = unittest.TestSuite() -# result = unittest.TestResult() -# for testSuite in suites: -# suite.addTest(unittest.makeSuite(testSuite)) -# runner = unittest.TextTestRunner() -# print(runner.run(suite)) -# runTests([TestBasicOperation]) From 2425b379b40b647c6bab7f5dfbe2bb9c6e299025 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sun, 5 Mar 2023 21:23:53 -0800 Subject: [PATCH 15/26] wip --- dbldatagen/schema_parser.py | 1 - 1 file changed, 1 deletion(-) diff --git a/dbldatagen/schema_parser.py b/dbldatagen/schema_parser.py index a42abc15..f3c0a561 100644 --- a/dbldatagen/schema_parser.py +++ b/dbldatagen/schema_parser.py @@ -132,7 +132,6 @@ def getTypeDefinitionParser(cls): pp.delimitedList(pp.Group(ident + pp.Optional(colon) + pp.Group(type_expr)))) + r_angle # try to capture invalid type name for better error reporting - # use `asKeyword` not `as_keyword` for maximum version compatibility invalid_type = pp.Word(pp.alphas, pp.alphanums+"_", asKeyword=True) # use left recursion to handle nesting of types From 16b27fd4559ee0b8921f3b814d26860640919424 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sun, 5 Mar 2023 21:25:25 -0800 Subject: [PATCH 16/26] wip --- tests/test_schema_parser.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/test_schema_parser.py b/tests/test_schema_parser.py index 0310a9ee..708d21d5 100644 --- a/tests/test_schema_parser.py +++ b/tests/test_schema_parser.py @@ -135,6 +135,7 @@ def test_table_definition_parser(self, setupLogging): ]) def test_sql_expression_cleanser(self, sqlExpr, expectedText): newSql = dg.SchemaParser._cleanseSQL(sqlExpr) + print(newSql) assert sqlExpr == expectedText or sqlExpr != newSql assert newSql == expectedText From 104de6b3d5a09f18acc48bcf9d84403fb2936a30 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sun, 5 Mar 2023 21:37:51 -0800 Subject: [PATCH 17/26] wip --- CHANGELOG.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index d354baa1..9ca2d6ba 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -12,8 +12,6 @@ All notable changes to the Databricks Labs Data Generator will be documented in * Adjusted column build phase separation (i.e which select statement is used to build columns) so that a column with a SQL expression can refer to previously created columns without use of a `baseColumn` attribute * Changed build labelling to comply with PEP440 -* Adjusted column build phase separation (i.e which select statement is used to build columns) so that a - column with a SQL expression can refer to previously created columns without use of a `baseColumn` attribute #### Fixed From d7f66bca4d3c6a9195df8a4a4d5e230b8738eb11 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sun, 5 Mar 2023 21:40:24 -0800 Subject: [PATCH 18/26] wip --- CHANGELOG.md | 9 --------- 1 file changed, 9 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9ca2d6ba..e26babc3 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -21,15 +21,6 @@ All notable changes to the Databricks Labs Data Generator will be documented in * Added support for complex event time in streaming scenarios. It may also be used in batch scenarios for testing * Parsing of SQL expressions to determine column dependencies -#### Notes -* This does not change actual order of column building - but adjusts which phase - - -#### Fixed - -#### Added -* Parsing of SQL expressions to determine column dependencies - #### Notes * This does not change actual order of column building - but adjusts which phase columns are built in From a2d98aeac6de3613f279684e4bae1b1c1a130c49 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Tue, 7 Mar 2023 07:43:13 -0800 Subject: [PATCH 19/26] wip --- tests/test_streaming.py | 52 ++++++++++++++++++++++------------------- 1 file changed, 28 insertions(+), 24 deletions(-) diff --git a/tests/test_streaming.py b/tests/test_streaming.py index 6443eb6a..f0c3c4f2 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -224,39 +224,43 @@ def test_basic_file_streaming(self, options, getStreamingDirs): # generate file for base of streaming generator testDataSpecBase = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, - partitions=4, seedMethod='hash_fieldname') + seedMethod='hash_fieldname') .withColumn('value', "long", expr="id") .withColumn("code1", IntegerType(), minValue=100, maxValue=200) .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) ) - dfBase = testDataSpecBase.build() - dfBase.write.format(options["dbldatagen.streaming.source"])\ - .mode('overwrite')\ - .save(options["dbldatagen.streaming.source"]) - # generate streaming data frame - testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set2", rows=self.row_count, - partitions=4, seedMethod='hash_fieldname') - .withColumn("a", IntegerType(), minValue=100, maxValue=200) - .withColumn("b", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) - ) - dfStreaming = testDataSpecBase.build(withStreaming=True, options=options) + pytest.skip("File based streaming not yet implemented") - sq = (dfStreaming - .writeStream - .format("parquet") - .outputMode("append") - .option("path", test_dir) - .option("checkpointLocation", checkpoint_dir) - .trigger(once=True) - .start()) + if False: + dfBase = testDataSpecBase.build() + dfBase.write.format(options["dbldatagen.streaming.source"])\ + .mode('overwrite')\ + .save(options["dbldatagen.streaming.source"]) - sq.processAllAvailable() + # generate streaming data frame + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set2", rows=self.row_count, + partitions=4, seedMethod='hash_fieldname') + .withColumn("a", IntegerType(), minValue=100, maxValue=200) + .withColumn("b", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + ) + dfStreaming = testDataSpecBase.build(withStreaming=True, options=options) - dfStreamDataRead = spark.read.format("parquet").load(test_dir) - rows_read = dfStreamDataRead.count() + sq = (dfStreaming + .writeStream + .format("parquet") + .outputMode("append") + .option("path", test_dir) + .option("checkpointLocation", checkpoint_dir) + .trigger(once=True) + .start()) + + sq.processAllAvailable() + + dfStreamDataRead = spark.read.format("parquet").load(test_dir) + rows_read = dfStreamDataRead.count() - assert rows_read == self.row_count + assert rows_read == self.row_count def test_withEventTime_batch(self): # test it in batch mode From b363acc68d7d3eb27ad44cec41eb6e75ad252a53 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Tue, 7 Mar 2023 15:39:50 -0800 Subject: [PATCH 20/26] wip --- tests/test_streaming.py | 29 +++++++++++++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/tests/test_streaming.py b/tests/test_streaming.py index f0c3c4f2..c05cef1a 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -313,4 +313,33 @@ def test_withEventTime_streaming(self, getStreamingDirs): rows_read = dfStreamDataRead.count() assert rows_read > 0 + @pytest.mark.parametrize("options,optionsExpected", + [ ({"dbldatagen.streaming.source": "rate"}, + ({"dbldatagen.streaming.source": "rate"}, + {"rowsPerSecond": 1, 'numPartitions': 10}, {})), + ({"dbldatagen.streaming.source": "rate-micro-batch"}, + ({"dbldatagen.streaming.source": "rate-micro-batch"}, {'numPartitions': 10, 'rowsPerBatch':1}, {})), + ]) + def test_default_options(self, options, optionsExpected): + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, + partitions=10, seedMethod='hash_fieldname') + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code2", IntegerType(), minValue=0, maxValue=10) + .withColumn("code3", StringType(), values=['a', 'b', 'c']) + .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + ) + + datagen_options, passthrough_options, unsupported_options = testDataSpec._parseBuildOptions(options) + testDataSpec._applyStreamingDefaults(datagen_options, passthrough_options) + if "startTimestamp" in passthrough_options.keys(): + passthrough_options.pop("startTimestamp", None) + + # remove start timestamp from both options and expected options + + expected_datagen_options, expected_passthrough_options, expected_unsupported_options = optionsExpected + + assert datagen_options == expected_datagen_options + assert passthrough_options == expected_passthrough_options + From 5f51db143dd2d539e930d0672a45e944d32f5b37 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Fri, 10 Mar 2023 10:42:52 -0800 Subject: [PATCH 21/26] wip --- dbldatagen/data_generator.py | 64 +++++++++++++++--- tests/test_streaming.py | 127 +++++++++++++++++++++++++++++++++++ 2 files changed, 180 insertions(+), 11 deletions(-) diff --git a/dbldatagen/data_generator.py b/dbldatagen/data_generator.py index 2608ab81..aee814f0 100644 --- a/dbldatagen/data_generator.py +++ b/dbldatagen/data_generator.py @@ -44,10 +44,11 @@ _STREAM_SOURCE_START_TIMESTAMP = "startTimestamp" _STREAMING_SOURCE_TEXT = "text" -_STREAMING_SOURCE_TEXT = "parquet" -_STREAMING_SOURCE_TEXT = "csv" -_STREAMING_SOURCE_TEXT = "json" -_STREAMING_SOURCE_TEXT = "ord" +_STREAMING_SOURCE_PARQUET = "parquet" +_STREAMING_SOURCE_CSV = "csv" +_STREAMING_SOURCE_JSON = "json" +_STREAMING_SOURCE_ORC = "ord" +_STREAMING_SOURCE_DELTA = "delta" class DataGenerator: """ Main Class for test data set generation @@ -901,19 +902,60 @@ def _getBaseDataFrame(self, startId=0, streaming=False, options=None): else: self._applyStreamingDefaults(build_options, passthrough_options) - status = ( - f"Generating streaming data frame with {id_partitions} partitions") + + assert _STREAMING_SOURCE_OPTION in build_options.keys(), "There must be a source type specified" + streaming_source_format = build_options[_STREAMING_SOURCE_OPTION] + + if streaming_source_format in [ _STREAMING_SOURCE_RATE, _STREAMING_SOURCE_RATE_MICRO_BATCH]: + streaming_partitions = passthrough_options[_STREAMING_SOURCE_NUM_PARTITIONS] + status = ( + f"Generating streaming data frame with {streaming_partitions} partitions") + else: + status = ( + f"Generating streaming data frame with '{streaming_source_format}' streaming source") + self.logger.info(status) self.executionHistory.append(status) df1 = (self.sparkSession.readStream - .format("rate")) + .format(streaming_source_format)) for k, v in passthrough_options.items(): df1 = df1.option(k, v) - df1 = (df1.load() - .withColumnRenamed("value", self._seedColumnName) - ) + + file_formats = [_STREAMING_SOURCE_TEXT, _STREAMING_SOURCE_JSON, _STREAMING_SOURCE_CSV, + _STREAMING_SOURCE_PARQUET, _STREAMING_SOURCE_DELTA, _STREAMING_SOURCE_ORC] + + data_path = None + source_table = None + id_column = "value" + + if _STREAMING_ID_FIELD_OPTION in build_options: + id_column = build_options[_STREAMING_ID_FIELD_OPTION] + + if _STREAMING_TABLE_OPTION in build_options: + source_table = build_options[_STREAMING_TABLE_OPTION] + + if _STREAMING_SCHEMA_OPTION in build_options: + source_schema = build_options[_STREAMING_SCHEMA_OPTION] + df1 = df1.schema(source_schema) + + # get path for file based reads + if _STREAMING_PATH_OPTION in build_options: + data_path = build_options[_STREAMING_PATH_OPTION] + elif streaming_source_format in file_formats: + if "path" in passthrough_options: + data_path = passthrough_options["path"] + + if data_path is not None: + df1 = df1.load(data_path) + elif source_table is not None: + df1 = df1.table(source_table) + else: + df1 = df1.load() + + if id_column != self._seedColumnName: + df1 = df1.withColumnRenamed(id_column, self._seedColumnName) return df1 @@ -1174,7 +1216,7 @@ def _applyStreamingDefaults(self, build_options, passthrough_options): build_options[_STREAMING_SOURCE_OPTION] = _STREAMING_SOURCE_RATE # setup `numPartitions` if not specified - if build_options[_STREAMING_SOURCE_OPTION] in [_STREAMING_SOURCE_RATE,_STREAMING_SOURCE_RATE_MICRO_BATCH]: + if build_options[_STREAMING_SOURCE_OPTION] in [_STREAMING_SOURCE_RATE, _STREAMING_SOURCE_RATE_MICRO_BATCH]: if _STREAMING_SOURCE_NUM_PARTITIONS not in passthrough_options: passthrough_options[_STREAMING_SOURCE_NUM_PARTITIONS] = self.partitions diff --git a/tests/test_streaming.py b/tests/test_streaming.py index c05cef1a..3c2c1879 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -2,20 +2,55 @@ import shutil import time import pytest +import logging from pyspark.sql.types import IntegerType, StringType, FloatType +import pyspark.sql.functions as F import dbldatagen as dg spark = dg.SparkSingleton.getLocalInstance("streaming tests") +@pytest.fixture(scope="class") +def setupLogging(): + FORMAT = '%(asctime)-15s %(message)s' + logging.basicConfig(format=FORMAT) + + class TestStreaming(): row_count = 100000 column_count = 10 time_to_run = 10 rows_per_second = 5000 + def setup_log_capture(self, caplog_object): + """ set up log capture fixture + + Sets up log capture fixture to only capture messages after setup and only + capture warnings and errors + + """ + caplog_object.set_level(logging.WARNING) + + # clear messages from setup + caplog_object.clear() + + def get_log_capture_warnings_and_errors(self, caplog_object, textFlag): + """ + gets count of errors containing specified text + + :param caplog_object: log capture object from fixture + :param textFlag: text to search for to include error or warning in count + :return: count of errors containg text specified in `textFlag` + """ + streaming_warnings_and_errors = 0 + for r in caplog_object.records: + if (r.levelname == "WARNING" or r.levelname == "ERROR") and textFlag in r.message: + streaming_warnings_and_errors += 1 + + return streaming_warnings_and_errors + @pytest.fixture def getStreamingDirs(self): time_now = int(round(time.time() * 1000)) @@ -32,6 +67,23 @@ def getStreamingDirs(self): shutil.rmtree(base_dir, ignore_errors=True) print(f"\n\n*** test dir [{base_dir}] deleted") + @pytest.fixture + def getDataDir(self): + time_now = int(round(time.time() * 1000)) + base_dir = "/tmp/testdata_{}".format(time_now) + data_dir = os.path.join(base_dir, "data") + print(f"test data dir created '{base_dir}'") + + # dont need to create the data dir + os.makedirs(base_dir) + + try: + yield data_dir + finally: + shutil.rmtree(base_dir, ignore_errors=True) + print(f"\n\n*** test data dir [{base_dir}] deleted") + + @pytest.mark.parametrize("seedColumnName", ["id", "_id", None]) @@ -342,4 +394,79 @@ def test_default_options(self, options, optionsExpected): assert datagen_options == expected_datagen_options assert passthrough_options == expected_passthrough_options + def test_text_streaming(self, getDataDir, caplog, getStreamingDirs): + datadir = getDataDir + base_dir, test_dir, checkpoint_dir = getStreamingDirs + + # caplog fixture captures log content + self.setup_log_capture(caplog) + + df = spark.range(10000).select(F.expr("cast(id as string)").alias("id")) + df.write.format("text").save(datadir) + + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set1", rows=self.row_count, + partitions=10, seedMethod='hash_fieldname') + .withColumn("code1", IntegerType(), minValue=100, maxValue=200) + .withColumn("code2", IntegerType(), minValue=0, maxValue=10) + .withColumn("code3", StringType(), values=['a', 'b', 'c']) + .withColumn("code4", StringType(), values=['a', 'b', 'c'], random=True) + .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + ) + + streamingOptions = { + 'dbldatagen.streaming.source': 'text', + 'dbldatagen.streaming.sourcePath': datadir, + + } + df_streaming = testDataSpec.build(withStreaming=True, options=streamingOptions) + + # check that there warnings about `text` format + text_format_warnings_and_errors = self.get_log_capture_warnings_and_errors(caplog, "text") + assert text_format_warnings_and_errors > 0, "Should have error or warning messages about text format" + + # loop until we get one seconds worth of data + start_time = time.time() + elapsed_time = 0 + rows_retrieved = 0 + time_limit = 10.0 + + while elapsed_time < time_limit and rows_retrieved < self.rows_per_second: + sq = (df_streaming + .writeStream + .format("parquet") + .outputMode("append") + .option("path", test_dir) + .option("checkpointLocation", checkpoint_dir) + .trigger(once=True) + .start()) + + # wait for trigger once to terminate + sq.awaitTermination(5) + + elapsed_time = time.time() - start_time + + try: + df2 = spark.read.format("parquet").load(test_dir) + rows_retrieved = df2.count() + + # ignore file or metadata not found issues arising from read before stream has written first batch + except Exception as exc: + print("Exception:", exc) + + if sq.isActive: + sq.stop() + + end_time = time.time() + + print("*** Done ***") + print("read {} rows from newly written data".format(rows_retrieved)) + print("elapsed time (seconds)", end_time - start_time) + + # check that we have at least one second of data + assert rows_retrieved >= self.rows_per_second + + + + + From bf005001ca7fb2540bd6e43d2297900103c4a4ec Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Fri, 10 Mar 2023 11:15:05 -0800 Subject: [PATCH 22/26] wip --- tests/test_streaming.py | 53 +++++++++++++++++++---------------------- 1 file changed, 25 insertions(+), 28 deletions(-) diff --git a/tests/test_streaming.py b/tests/test_streaming.py index 3c2c1879..424fa181 100644 --- a/tests/test_streaming.py +++ b/tests/test_streaming.py @@ -282,37 +282,34 @@ def test_basic_file_streaming(self, options, getStreamingDirs): .withColumn("code5", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) ) - pytest.skip("File based streaming not yet implemented") - - if False: - dfBase = testDataSpecBase.build() - dfBase.write.format(options["dbldatagen.streaming.source"])\ - .mode('overwrite')\ - .save(options["dbldatagen.streaming.source"]) - - # generate streaming data frame - testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set2", rows=self.row_count, - partitions=4, seedMethod='hash_fieldname') - .withColumn("a", IntegerType(), minValue=100, maxValue=200) - .withColumn("b", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) - ) - dfStreaming = testDataSpecBase.build(withStreaming=True, options=options) - - sq = (dfStreaming - .writeStream - .format("parquet") - .outputMode("append") - .option("path", test_dir) - .option("checkpointLocation", checkpoint_dir) - .trigger(once=True) - .start()) + dfBase = testDataSpecBase.build() + dfBase.write.format(options["dbldatagen.streaming.source"])\ + .mode('overwrite')\ + .save(options["dbldatagen.streaming.source"]) + + # generate streaming data frame + testDataSpec = (dg.DataGenerator(sparkSession=spark, name="test_data_set2", rows=self.row_count, + partitions=4, seedMethod='hash_fieldname') + .withColumn("a", IntegerType(), minValue=100, maxValue=200) + .withColumn("b", StringType(), values=['a', 'b', 'c'], random=True, weights=[9, 1, 1]) + ) + dfStreaming = testDataSpecBase.build(withStreaming=True, options=options) - sq.processAllAvailable() + sq = (dfStreaming + .writeStream + .format("parquet") + .outputMode("append") + .option("path", test_dir) + .option("checkpointLocation", checkpoint_dir) + .trigger(once=True) + .start()) + + sq.processAllAvailable() - dfStreamDataRead = spark.read.format("parquet").load(test_dir) - rows_read = dfStreamDataRead.count() + dfStreamDataRead = spark.read.format("parquet").load(test_dir) + rows_read = dfStreamDataRead.count() - assert rows_read == self.row_count + assert rows_read == self.row_count def test_withEventTime_batch(self): # test it in batch mode From c24aa4fb9731e3648c8d5ce7028d77a7718dad06 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sat, 11 Mar 2023 01:06:12 -0800 Subject: [PATCH 23/26] merge from master --- .github/workflows/onrelease.yml | 3 +++ .github/workflows/push.yml | 3 +++ CHANGELOG.md | 6 ++++-- CONTRIBUTING.md | 3 ++- Pipfile | 19 +++++++++++-------- README.md | 17 ++++++++++++++++- dbldatagen/_version.py | 2 +- dbldatagen/schema_parser.py | 8 ++++---- docs/source/conf.py | 2 +- python/.bumpversion.cfg | 2 +- python/dev_require.txt | 8 ++++---- python/require.txt | 6 +++--- setup.py | 2 +- 13 files changed, 54 insertions(+), 27 deletions(-) diff --git a/.github/workflows/onrelease.yml b/.github/workflows/onrelease.yml index 3a734743..479a0cbf 100644 --- a/.github/workflows/onrelease.yml +++ b/.github/workflows/onrelease.yml @@ -30,6 +30,9 @@ jobs: - name: Install run: pip install pipenv + - name: Install dependencies + run: pipenv install --dev + - name: Build dist run: pipenv run python setup.py sdist bdist_wheel diff --git a/.github/workflows/push.yml b/.github/workflows/push.yml index e3cec53d..b9ac6080 100644 --- a/.github/workflows/push.yml +++ b/.github/workflows/push.yml @@ -37,6 +37,9 @@ jobs: - name: Install run: pip install pipenv + - name: Install dependencies + run: pipenv install --dev + - name: Run tests run: make test diff --git a/CHANGELOG.md b/CHANGELOG.md index e26babc3..640def3f 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,7 +3,7 @@ ## Change History All notable changes to the Databricks Labs Data Generator will be documented in this file. -### Unreleased +### Version 0.3.2 #### Changed * Additional migration of tests to use of `pytest` @@ -14,6 +14,7 @@ All notable changes to the Databricks Labs Data Generator will be documented in * Changed build labelling to comply with PEP440 #### Fixed +* Fixed compatibility of build with older versions of runtime that rely on `pyparsing` version 2.4.7 #### Added * Added support for additional streaming source types and for use of custom streaming sources @@ -22,7 +23,8 @@ All notable changes to the Databricks Labs Data Generator will be documented in * Parsing of SQL expressions to determine column dependencies #### Notes -* This does not change actual order of column building - but adjusts which phase columns are built in +* The enhancements to build ordering does not change actual order of column building - + but adjusts which phase columns are built in ### Version 0.3.1 diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 3cf2565b..2d20cbd6 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -14,7 +14,8 @@ warrant that you have the legal authority to do so. # Building the code ## Package Dependencies -See the contents of the file `python/require.txt` to see the Python package dependencies +See the contents of the file `python/require.txt` to see the Python package dependencies. +Dependent packages are not installed automatically by the `dbldatagen` package. ## Python compatibility diff --git a/Pipfile b/Pipfile index a98bdb1e..b3ea65f4 100644 --- a/Pipfile +++ b/Pipfile @@ -6,13 +6,6 @@ verify_ssl = true [dev-packages] pytest = "*" pytest-cov = "*" - -numpy = "1.22.0" -pyspark = "3.1.3" -pyarrow = "1.0.1" -pandas = "1.1.3" -pyparsing = ">=2.4.7,<3.0.9" - sphinx = ">=2.0.0,<3.1.0" nbsphinx = "*" numpydoc = "0.8" @@ -21,6 +14,16 @@ ipython = "7.31.1" pydata-sphinx-theme = "*" recommonmark = "*" sphinx-markdown-builder = "*" +bumpversion = "*" + +[packages] +numpy = "==1.22.0" +pyspark = "==3.1.3" +pyarrow = "==4.0.1" +wheel = "==0.38.4" +pandas = "==1.2.4" +setuptools = "==65.6.3" +pyparsing = "==2.4.7" [requires] -python_version = "3.8" +python_version = ">=3.8.10" diff --git a/README.md b/README.md index 49f517ff..2e8ff2ab 100644 --- a/README.md +++ b/README.md @@ -60,7 +60,7 @@ details of use and many examples. Release notes and details of the latest changes for this specific release can be found in the Github repository -[here](https://github.com/databrickslabs/dbldatagen/blob/release/v0.3.2a0/CHANGELOG.md) +[here](https://github.com/databrickslabs/dbldatagen/blob/release/v0.3.2/CHANGELOG.md) # Installation @@ -126,6 +126,21 @@ examples. The Github repository also contains further examples in the examples directory +## Spark and Databricks Runtime Compatibility +The `dbldatagen` package is intended to be compatible with recent LTS versions of the Databricks runtime including +older LTS versions at least from 10.4 LTS and later. It also aims to be compatible with Delta Live Table runtimes +including `current` and `preview`. + +While we dont specifically drop support for older runtimes, changes in Pyspark APIs or +APIs from dependent packages such as `numpy`, `pandas`, `pyarrow` and `pyparsing` make cause issues with older +runtimes. + +Installing `dbldatagen` explicitly does not install releases of dependent packages so as to preserve the curated +set of packages installed in any Databricks runtime environment. + +When building on local environments, the `Pipfile` and requirements files are used to determine the versions +tested against for releases and unit tests. + ## Project Support Please note that all projects released under [`Databricks Labs`](https://www.databricks.com/learn/labs) are provided for your exploration only, and are not formally supported by Databricks with Service Level Agreements diff --git a/dbldatagen/_version.py b/dbldatagen/_version.py index 9a08822f..0fdb74ce 100644 --- a/dbldatagen/_version.py +++ b/dbldatagen/_version.py @@ -33,7 +33,7 @@ def get_version(version): return version_info -__version__ = "0.3.2a0" # DO NOT EDIT THIS DIRECTLY! It is managed by bumpversion +__version__ = "0.3.2" # DO NOT EDIT THIS DIRECTLY! It is managed by bumpversion __version_info__ = get_version(__version__) diff --git a/dbldatagen/schema_parser.py b/dbldatagen/schema_parser.py index f3c0a561..88751e98 100644 --- a/dbldatagen/schema_parser.py +++ b/dbldatagen/schema_parser.py @@ -273,18 +273,18 @@ def _cleanseSQL(cls, sql_string): # skip over quoted identifiers even if they contain quotes quoted_ident = pp.QuotedString(quoteChar="`", escQuote="``") - quoted_ident.set_parse_action(lambda s, loc, toks: f"`{toks[0]}`") + quoted_ident.setParseAction(lambda s, loc, toks: f"`{toks[0]}`") stringForm1 = pp.Literal('r') + pp.QuotedString(quoteChar="'") stringForm2 = pp.Literal('r') + pp.QuotedString(quoteChar='"') stringForm3 = pp.QuotedString(quoteChar="'", escQuote=r"\'") stringForm4 = pp.QuotedString(quoteChar='"', escQuote=r'\"') stringForm = stringForm1 ^ stringForm2 ^ stringForm3 ^ stringForm4 - stringForm.set_parse_action(lambda s, loc, toks: "' '") + stringForm.setParseAction(lambda s, loc, toks: "' '") parser = quoted_ident ^ stringForm - transformed_string = parser.transform_string(sql_string) + transformed_string = parser.transformString(sql_string) return transformed_string @@ -312,7 +312,7 @@ def columnsReferencesFromSQLString(cls, sql_string, filter=None): ident = pp.Word(pp.alphas, pp.alphanums + "_") | pp.QuotedString(quoteChar="`", escQuote="``") parser = ident - references = parser.search_string(cleansed_sql_string) + references = parser.searchString(cleansed_sql_string) results = set([item for sublist in references for item in sublist]) diff --git a/docs/source/conf.py b/docs/source/conf.py index 86b1a7a7..e5d97ce9 100644 --- a/docs/source/conf.py +++ b/docs/source/conf.py @@ -28,7 +28,7 @@ author = 'Databricks Inc' # The full version, including alpha/beta/rc tags -release = "0.3.2a0" # DO NOT EDIT THIS DIRECTLY! It is managed by bumpversion +release = "0.3.2" # DO NOT EDIT THIS DIRECTLY! It is managed by bumpversion # -- General configuration --------------------------------------------------- diff --git a/python/.bumpversion.cfg b/python/.bumpversion.cfg index 7681b46c..a0db78a6 100644 --- a/python/.bumpversion.cfg +++ b/python/.bumpversion.cfg @@ -1,5 +1,5 @@ [bumpversion] -current_version = 0.3.2a0 +current_version = 0.3.2 commit = False tag = False parse = (?P\d+)\.(?P\d+)\.(?P\d+){0,1}(?P\D*)(?P\d*) diff --git a/python/dev_require.txt b/python/dev_require.txt index c6a3737d..362cf2a0 100644 --- a/python/dev_require.txt +++ b/python/dev_require.txt @@ -1,14 +1,14 @@ # The following packages are used in building the test data generator framework. # All packages used are already installed in the Databricks runtime environment for version 6.5 or later -numpy==1.19.2 +numpy==1.22.0 pandas==1.2.4 pickleshare==0.7.5 py4j==0.10.9 -pyarrow==4.0.0 -pyspark>=3.1.2 +pyarrow==4.0.1 +pyspark>=3.1.3 python-dateutil==2.8.1 six==1.15.0 -pyparsing>=2.4.7, <= 3.0.9 +pyparsing==2.4.7 # The following packages are required for development only wheel==0.36.2 diff --git a/python/require.txt b/python/require.txt index 0f94f990..9a31dd5e 100644 --- a/python/require.txt +++ b/python/require.txt @@ -4,11 +4,11 @@ numpy==1.22.0 pandas==1.2.5 pickleshare==0.7.5 py4j==0.10.9 -pyarrow==4.0.0 -pyspark>=3.1.2 +pyarrow==4.0.1 +pyspark>=3.1.3 python-dateutil==2.8.1 six==1.15.0 -pyparsing>=2.4.7, <= 3.0.9 +pyparsing==2.4.7 # The following packages are required for development only wheel==0.36.2 diff --git a/setup.py b/setup.py index 46c6196f..d3a932f9 100644 --- a/setup.py +++ b/setup.py @@ -31,7 +31,7 @@ setuptools.setup( name="dbldatagen", - version="0.3.2a0", + version="0.3.2", author="Ronan Stokes, Databricks", description="Databricks Labs - PySpark Synthetic Data Generator", long_description=long_description, From 85a0312a11f93c53270dff7ef9f2d5ef1ac5cc3e Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sat, 1 Jul 2023 17:32:11 -0700 Subject: [PATCH 24/26] merged from master --- .github/workflows/onrelease.yml | 13 ++++++++----- .github/workflows/push.yml | 27 ++++++++++++++++++--------- 2 files changed, 26 insertions(+), 14 deletions(-) diff --git a/.github/workflows/onrelease.yml b/.github/workflows/onrelease.yml index 479a0cbf..3827a3d6 100644 --- a/.github/workflows/onrelease.yml +++ b/.github/workflows/onrelease.yml @@ -18,15 +18,18 @@ jobs: - name: Checkout uses: actions/checkout@v2 - - name: Unshallow - run: git fetch --prune --unshallow - - name: Set up Python 3.8 - uses: actions/setup-python@v3 + uses: actions/setup-python@v4 with: - python-version: '3.8' + python-version: '3.8.12' cache: 'pipenv' + - name: Check Python version + run: python --version + + - name: Install pip + run: python -m pip install --upgrade pip + - name: Install run: pip install pipenv diff --git a/.github/workflows/push.yml b/.github/workflows/push.yml index b9ac6080..5e9768c4 100644 --- a/.github/workflows/push.yml +++ b/.github/workflows/push.yml @@ -8,14 +8,13 @@ on: jobs: tests: - # Ubuntu latest no longer installs Python 3.8 by default so install it + # Ubuntu latest no longer installs Python 3.9 by default so install it runs-on: ubuntu-latest steps: - name: Checkout - uses: actions/checkout@v2 - - - name: Unshallow - run: git fetch --prune --unshallow + uses: actions/checkout@v3 + with: + fetch-depth: 0 # - name: Cache packages # uses: actions/cache@v2 @@ -29,23 +28,33 @@ jobs: # ${{ runner.os }}-go- - name: Set up Python 3.8 - uses: actions/setup-python@v3 + uses: actions/setup-python@v4 with: - python-version: '3.8' + python-version: '3.8.12' cache: 'pipenv' + - name: Check Python version + run: python --version + + - name: Install pip + run: python -m pip install --upgrade pip + - name: Install run: pip install pipenv - name: Install dependencies run: pipenv install --dev + - name: Lint + run: | + pipenv run prospector --profile prospector.yaml + - name: Run tests run: make test - name: Publish test coverage to coverage site - uses: codecov/codecov-action@v2 + uses: codecov/codecov-action@v1 with: files: ./coverage.xml name: dbldatagen - verbose: true \ No newline at end of file + verbose: true From 4ac18c95841465def0f0893a48d1f502442bbc11 Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sat, 1 Jul 2023 17:48:06 -0700 Subject: [PATCH 25/26] changes to changelog --- CHANGELOG.md | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index cd157e46..72c65fcf 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,11 +3,20 @@ ## Change History All notable changes to the Databricks Labs Data Generator will be documented in this file. -### Version 0.3.2 +### Version 0.3.5 #### Changed * Added formatting of generated code as Html for script methods * Modified pipfile to use newer version of package specifications +* Additional migration of tests to use of `pytest` +* Changed parsing of build options for data generator to support use of custom streaming +* Documentation updates in support of new features such as streaming, complex structures etc + +#### Added +* Added support for additional streaming source types and for use of custom streaming sources +* Added support for use of file reads as a streaming source (for seed and timestamp columns only) +* Added support for complex event time in streaming scenarios. It may also be used in batch scenarios for testing + ### Version 0.3.4 Post 3 @@ -60,9 +69,6 @@ Thanks to Marvin Schenkel for the contribution ### Version 0.3.2 #### Changed -* Additional migration of tests to use of `pytest` -* Changed parsing of build options for data generator to support use of custom streaming -* Documentation updates in support of new features such as streaming, complex structures etc * Adjusted column build phase separation (i.e which select statement is used to build columns) so that a column with a SQL expression can refer to previously created columns without use of a `baseColumn` attribute * Changed build labelling to comply with PEP440 @@ -71,9 +77,6 @@ Thanks to Marvin Schenkel for the contribution * Fixed compatibility of build with older versions of runtime that rely on `pyparsing` version 2.4.7 #### Added -* Added support for additional streaming source types and for use of custom streaming sources -* Added support for use of file reads as a streaming source (for seed and timestamp columns only) -* Added support for complex event time in streaming scenarios. It may also be used in batch scenarios for testing * Parsing of SQL expressions to determine column dependencies #### Notes From 041b3dcacb2374cf507d922693a25c730e61b32b Mon Sep 17 00:00:00 2001 From: ronanstokes-db Date: Sat, 1 Jul 2023 17:50:45 -0700 Subject: [PATCH 26/26] changes to changelog --- README.md | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/README.md b/README.md index 0c0d636a..645da821 100644 --- a/README.md +++ b/README.md @@ -152,21 +152,6 @@ to preserve the curated set of packages pre-installed in any Databricks runtime When building on local environments, the build process uses the `Pipfile` and requirements files to determine the package versions for releases and unit tests. -## Spark and Databricks Runtime Compatibility -The `dbldatagen` package is intended to be compatible with recent LTS versions of the Databricks runtime including -older LTS versions at least from 10.4 LTS and later. It also aims to be compatible with Delta Live Table runtimes -including `current` and `preview`. - -While we dont specifically drop support for older runtimes, changes in Pyspark APIs or -APIs from dependent packages such as `numpy`, `pandas`, `pyarrow` and `pyparsing` make cause issues with older -runtimes. - -Installing `dbldatagen` explicitly does not install releases of dependent packages so as to preserve the curated -set of packages installed in any Databricks runtime environment. - -When building on local environments, the `Pipfile` and requirements files are used to determine the versions -tested against for releases and unit tests. - ## Project Support Please note that all projects released under [`Databricks Labs`](https://www.databricks.com/learn/labs) are provided for your exploration only, and are not formally supported by Databricks with Service Level Agreements