From 66f83ad5c344f3586134f20a5a6ea481a4341227 Mon Sep 17 00:00:00 2001 From: GBBBAS <42962356+GBBBAS@users.noreply.github.com> Date: Fri, 4 Aug 2023 17:09:14 +0100 Subject: [PATCH 1/3] Add Trigger Logic Signed-off-by: GBBBAS <42962356+GBBBAS@users.noreply.github.com> --- .../rtdip_sdk/pipelines/deploy/databricks.py | 2 ++ .../pipelines/destinations/spark/delta.py | 7 ++++--- .../pipelines/destinations/spark/delta_merge.py | 5 +++-- .../pipelines/destinations/spark/eventhub.py | 13 +++++++++---- .../pipelines/destinations/spark/kafka.py | 12 ++++++++---- .../pipelines/destinations/spark/kinesis.py | 14 +++++++------- .../pipelines/destinations/spark/pcdm_to_delta.py | 7 ++++--- .../pipelines/destinations/spark/rest_api.py | 5 +++-- 8 files changed, 40 insertions(+), 25 deletions(-) diff --git a/src/sdk/python/rtdip_sdk/pipelines/deploy/databricks.py b/src/sdk/python/rtdip_sdk/pipelines/deploy/databricks.py index 7d560d05d..851dca269 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/deploy/databricks.py +++ b/src/sdk/python/rtdip_sdk/pipelines/deploy/databricks.py @@ -122,12 +122,14 @@ def deploy(self) -> Union[bool, ValueError]: if task.notebook_task is not None: module = self._load_module(task.task_key + "file_upload", task.notebook_task.notebook_path) (task_libraries, spark_configuration) = PipelineComponentsGetUtility(module.__name__).execute() + workspace_client.workspace.mkdirs(path=self.workspace_directory) path="{}/{}".format(self.workspace_directory, Path(task.notebook_task.notebook_path).name) workspace_client.workspace.upload(path=path, overwrite=True, content=self._convert_file_to_binary(task.notebook_task.notebook_path)) task.notebook_task.notebook_path = path else: module = self._load_module(task.task_key + "file_upload", task.spark_python_task.python_file) (task_libraries, spark_configuration) = PipelineComponentsGetUtility(module).execute() + workspace_client.workspace.mkdirs(path=self.workspace_directory) path="{}/{}".format(self.workspace_directory, Path(task.spark_python_task.python_file).name) workspace_client.workspace.upload(path=path, overwrite=True, content=self._convert_file_to_binary(task.spark_python_task.python_file)) task.spark_python_task.python_file = path diff --git a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/delta.py b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/delta.py index e42720f31..b4e0847b5 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/delta.py +++ b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/delta.py @@ -30,7 +30,7 @@ class SparkDeltaDestination(DestinationInterface): options (dict): Options that can be specified for a Delta Table write operation (See Attributes table below). Further information on the options is available for [batch](https://docs.delta.io/latest/delta-batch.html#write-to-a-table){ target="_blank" } and [streaming](https://docs.delta.io/latest/delta-streaming.html#delta-table-as-a-sink){ target="_blank" }. destination (str): Either the name of the Hive Metastore or Unity Catalog Delta Table **or** the path to the Delta table mode (str): Method of writing to Delta Table - append/overwrite (batch), append/complete (stream) - trigger (str): Frequency of the write operation + trigger (str): Frequency of the write operation. Specify "availableNow" to execute a trigger once, otherwise specify a time period such as "30 seconds", "5 minutes" query_name (str): Unique name for the query in associated SparkSession Attributes: @@ -119,12 +119,13 @@ def write_stream(self): ''' Writes streaming data to Delta. Exactly-once processing is guaranteed ''' + TRIGGER_OPTION = {'availableNow': True} if self.trigger == "availableNow" else {'processingTime': self.trigger} try: if "/" in self.destination: query = ( self.data .writeStream - .trigger(processingTime=self.trigger) + .trigger(**TRIGGER_OPTION) .format("delta") .queryName(self.query_name) .outputMode(self.mode) @@ -135,7 +136,7 @@ def write_stream(self): query = ( self.data .writeStream - .trigger(processingTime=self.trigger) + .trigger(**TRIGGER_OPTION) .format("delta") .queryName(self.query_name) .outputMode(self.mode) diff --git a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/delta_merge.py b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/delta_merge.py index 093a06201..127aaae96 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/delta_merge.py +++ b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/delta_merge.py @@ -48,7 +48,7 @@ class SparkDeltaMergeDestination(DestinationInterface): when_not_matched_by_source_update_list (list[DeltaMergeConditionValues]): Conditions(optional) and values to be used when updating rows that do not match the `merge_condition`. when_not_matched_by_source_delete_list (list[DeltaMergeCondition]): Conditions(optional) to be used when deleting rows that do not match the `merge_condition`. try_broadcast_join (bool): Attempts to perform a broadcast join in the merge which can leverage data skipping using partition pruning and file pruning automatically. Can fail if dataframe being merged is large and therefore more suitable for streaming merges than batch merges - trigger (str): Frequency of the write operation + trigger (str): Frequency of the write operation. Specify "availableNow" to execute a trigger once, otherwise specify a time period such as "30 seconds", "5 minutes" query_name (str): Unique name for the query in associated SparkSession Attributes: @@ -226,11 +226,12 @@ def write_stream(self): ''' Merges streaming data to Delta using foreachBatch ''' + TRIGGER_OPTION = {'availableNow': True} if self.trigger == "availableNow" else {'processingTime': self.trigger} try: query = ( self.data .writeStream - .trigger(processingTime=self.trigger) + .trigger(**TRIGGER_OPTION) .format("delta") .foreachBatch(self._stream_merge_micro_batch) .queryName(self.query_name) diff --git a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/eventhub.py b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/eventhub.py index ed98f535c..1e927c712 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/eventhub.py +++ b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/eventhub.py @@ -29,7 +29,9 @@ class SparkEventhubDestination(DestinationInterface): Args: data (DataFrame): Dataframe to be written to Eventhub options (dict): A dictionary of Eventhub configurations (See Attributes table below). All Configuration options for Eventhubs can be found [here.](https://github.com/Azure/azure-event-hubs-spark/blob/master/docs/PySpark/structured-streaming-pyspark.md#event-hubs-configuration){ target="_blank" } - + trigger (str): Frequency of the write operation. Specify "availableNow" to execute a trigger once, otherwise specify a time period such as "30 seconds", "5 minutes" + query_name (str): Unique name for the query in associated SparkSession + Attributes: checkpointLocation (str): Path to checkpoint files. (Streaming) eventhubs.connectionString (str): Eventhubs connection string is required to connect to the Eventhubs service. (Streaming and Batch) @@ -38,12 +40,12 @@ class SparkEventhubDestination(DestinationInterface): eventhubs.endingPosition: (JSON str): The ending position of a batch query. This works the same as startingPosition. (Batch) maxEventsPerTrigger (long): Rate limit on maximum number of events processed per trigger interval. The specified total number of events will be proportionally split across partitions of different volume. (Stream) ''' - data: DataFrame - options: dict - def __init__(self, data: DataFrame, options: dict) -> None: + def __init__(self, data: DataFrame, options: dict, trigger="10 seconds", query_name="EventhubDestination") -> None: self.data = data self.options = options + self.trigger = trigger + self.query_name = query_name @staticmethod def system_type(): @@ -94,11 +96,14 @@ def write_stream(self): Writes steaming data to Eventhubs. ''' try: + TRIGGER_OPTION = {'availableNow': True} if self.trigger == "availableNow" else {'processingTime': self.trigger} query = ( self.data .writeStream + .trigger(**TRIGGER_OPTION) .format("eventhubs") .options(**self.options) + .queryName(self.query_name) .start() ) while query.isActive: diff --git a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/kafka.py b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/kafka.py index 7f516d8af..f20b1ce23 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/kafka.py +++ b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/kafka.py @@ -33,6 +33,8 @@ class SparkKafkaDestination(DestinationInterface): Args: data (DataFrame): Dataframe to be written to Kafka options (dict): A dictionary of Kafka configurations (See Attributes tables below). For more information on configuration options see [here](https://spark.apache.org/docs/latest/structured-streaming-kafka-integration.html){ target="_blank" } + trigger (str): Frequency of the write operation. Specify "availableNow" to execute a trigger once, otherwise specify a time period such as "30 seconds", "5 minutes" + query_name (str): Unique name for the query in associated SparkSession The following options must be set for the Kafka destination for both batch and streaming queries. @@ -46,12 +48,11 @@ class SparkKafkaDestination(DestinationInterface): includeHeaders (bool): Whether to include the Kafka headers in the row. (Streaming and Batch) ''' - data: DataFrame - options: dict - - def __init__(self, data: DataFrame, options: dict) -> None: + def __init__(self, data: DataFrame, options: dict, trigger="10 seconds", query_name="KafkaDestination") -> None: self.data = data self.options = options + self.trigger = trigger + self.query_name = query_name @staticmethod def system_type(): @@ -103,12 +104,15 @@ def write_stream(self): Writes steaming data to Kafka. ''' try: + TRIGGER_OPTION = {'availableNow': True} if self.trigger == "availableNow" else {'processingTime': self.trigger} query = ( self.data .select(to_json(struct("*")).alias("value")) .writeStream + .trigger(**TRIGGER_OPTION) .format("kafka") .options(**self.options) + .queryName(self.query_name) .start() ) while query.isActive: diff --git a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/kinesis.py b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/kinesis.py index d1f2b8c4a..d6743ad66 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/kinesis.py +++ b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/kinesis.py @@ -26,7 +26,8 @@ class SparkKinesisDestination(DestinationInterface): data (DataFrame): Dataframe to be written to Delta options (dict): A dictionary of Kinesis configurations (See Attributes table below). All Configuration options for Kinesis can be found [here.](https://github.com/qubole/kinesis-sql#kinesis-sink-configuration){ target="_blank" } mode (str): Method of writing to Kinesis - append, complete, update - trigger (str): Frequency of the write operation + trigger (str): Frequency of the write operation. Specify "availableNow" to execute a trigger once, otherwise specify a time period such as "30 seconds", "5 minutes" + query_name (str): Unique name for the query in associated SparkSession Attributes: endpointUrl (str): Endpoint of the kinesis stream. @@ -34,15 +35,12 @@ class SparkKinesisDestination(DestinationInterface): awsSecretKey (str): AWS secret access key corresponding to the access key. streamName (List[str]): Name of the streams in Kinesis to write to. ''' - options: dict - mode: str - trigger: str - - def __init__(self, data: DataFrame, options: dict, mode:str = "update", trigger:str= "10 seconds") -> None: + def __init__(self, data: DataFrame, options: dict, mode:str = "update", trigger:str= "10 seconds", query_name="KinesisDestination") -> None: self.data = data self.options = options self.mode = mode self.trigger = trigger + self.query_name = query_name @staticmethod def system_type(): @@ -91,13 +89,15 @@ def write_stream(self): Writes steaming data to Kinesis. ''' try: + TRIGGER_OPTION = {'availableNow': True} if self.trigger == "availableNow" else {'processingTime': self.trigger} query = ( self.data .writeStream - .trigger(processingTime=self.trigger) + .trigger(**TRIGGER_OPTION) .format("kinesis") .outputMode(self.mode) .options(**self.options) + .queryName(self.query_name) .start() ) while query.isActive: diff --git a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/pcdm_to_delta.py b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/pcdm_to_delta.py index f282b78a2..ce36b921d 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/pcdm_to_delta.py +++ b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/pcdm_to_delta.py @@ -41,7 +41,7 @@ class SparkPCDMToDeltaDestination(DestinationInterface): destination_string (str): Either the name of the Hive Metastore or Unity Catalog Delta Table **or** the path to the Delta table to store string values. destination_integer (Optional str): Either the name of the Hive Metastore or Unity Catalog Delta Table **or** the path to the Delta table to store integer values mode (str): Method of writing to Delta Table - append/overwrite (batch), append/complete (stream) - trigger (str): Frequency of the write operation + trigger (str): Frequency of the write operation. Specify "availableNow" to execute a trigger once, otherwise specify a time period such as "30 seconds", "5 minutes" query_name (str): Unique name for the query in associated SparkSession merge (bool): Use Delta Merge to perform inserts, updates and deletes try_broadcast_join (bool): Attempts to perform a broadcast join in the merge which can leverage data skipping using partition pruning and file pruning automatically. Can fail if dataframe being merged is large and therefore more suitable for streaming merges than batch merges @@ -74,7 +74,7 @@ def __init__(self, destination_integer: str = None, mode: str = None, trigger="10 seconds", - query_name: str ="PCDMToDeltaMergeDestination", + query_name: str ="PCDMToDeltaDestination", merge: bool = True, try_broadcast_join = False, remove_nanoseconds: bool = False, @@ -247,11 +247,12 @@ def write_stream(self): Writes streaming Process Control Data Model data to Delta using foreachBatch ''' try: + TRIGGER_OPTION = {'availableNow': True} if self.trigger == "availableNow" else {'processingTime': self.trigger} if self.merge == True: query = ( self.data .writeStream - .trigger(processingTime=self.trigger) + .trigger(**TRIGGER_OPTION) .format("delta") .foreachBatch(self._write_stream_microbatches) .queryName(self.query_name) diff --git a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/rest_api.py b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/rest_api.py index 2dec9abdf..f7f46f3e6 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/rest_api.py +++ b/src/sdk/python/rtdip_sdk/pipelines/destinations/spark/rest_api.py @@ -44,7 +44,7 @@ class SparkRestAPIDestination(DestinationInterface): batch_size (int): The number of DataFrame rows to be used in each Rest API call method (str): The method to be used when calling the Rest API. Allowed values are POST, PATCH and PUT parallelism (int): The number of concurrent calls to be made to the Rest API - trigger (str): Frequency of the write operation + trigger (str): Frequency of the write operation. Specify "availableNow" to execute a trigger once, otherwise specify a time period such as "30 seconds", "5 minutes" query_name (str): Unique name for the query in associated SparkSession Attributes: @@ -172,10 +172,11 @@ def write_stream(self): Writes streaming data to a Rest API ''' try: + TRIGGER_OPTION = {'availableNow': True} if self.trigger == "availableNow" else {'processingTime': self.trigger} query = ( self.data .writeStream - .trigger(processingTime=self.trigger) + .trigger(**TRIGGER_OPTION) .foreachBatch(self._api_micro_batch) .queryName(self.query_name) .outputMode("update") From a80bee8f9dd3c7fb33820a1e3513f57c016fdecd Mon Sep 17 00:00:00 2001 From: GBBBAS <42962356+GBBBAS@users.noreply.github.com> Date: Fri, 4 Aug 2023 17:13:04 +0100 Subject: [PATCH 2/3] Update documentation Signed-off-by: GBBBAS <42962356+GBBBAS@users.noreply.github.com> --- .../python/rtdip_sdk/pipelines/sources/spark/autoloader.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/sdk/python/rtdip_sdk/pipelines/sources/spark/autoloader.py b/src/sdk/python/rtdip_sdk/pipelines/sources/spark/autoloader.py index 3f54b41ca..e05910704 100644 --- a/src/sdk/python/rtdip_sdk/pipelines/sources/spark/autoloader.py +++ b/src/sdk/python/rtdip_sdk/pipelines/sources/spark/autoloader.py @@ -66,9 +66,9 @@ def post_read_validation(self, df: DataFrame): def read_batch(self): ''' Raises: - NotImplementedError: Auto Loader only supports streaming reads. To perform a batch read, use the read_stream method of this component and specify the Trigger on the write_stream to be `availableNow=True` to perform batch-like reads of cloud storage files. + NotImplementedError: Auto Loader only supports streaming reads. To perform a batch read, use the read_stream method of this component and specify the Trigger on the write_stream to be `availableNow` to perform batch-like reads of cloud storage files. ''' - raise NotImplementedError("Auto Loader only supports streaming reads. To perform a batch read, use the read_stream method and specify Trigger on the write_stream as `availableNow=True`") + raise NotImplementedError("Auto Loader only supports streaming reads. To perform a batch read, use the read_stream method and specify Trigger on the write_stream as `availableNow`") def read_stream(self) -> DataFrame: ''' From a484b6503b21b8631c5020ba10f873e3afbce425 Mon Sep 17 00:00:00 2001 From: GBBBAS <42962356+GBBBAS@users.noreply.github.com> Date: Fri, 4 Aug 2023 18:40:30 +0100 Subject: [PATCH 3/3] Update of tests Signed-off-by: GBBBAS <42962356+GBBBAS@users.noreply.github.com> --- .../rtdip_sdk/pipelines/deploy/test_databricks_deploy.py | 2 ++ .../rtdip_sdk/pipelines/destinations/spark/test_eventhub.py | 4 ++-- .../rtdip_sdk/pipelines/destinations/spark/test_kafka.py | 4 ++-- .../rtdip_sdk/pipelines/destinations/spark/test_kinesis.py | 6 +++--- .../rtdip_sdk/pipelines/sources/spark/test_autoloader.py | 2 +- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/tests/sdk/python/rtdip_sdk/pipelines/deploy/test_databricks_deploy.py b/tests/sdk/python/rtdip_sdk/pipelines/deploy/test_databricks_deploy.py index 2b2e6b26b..04e405969 100644 --- a/tests/sdk/python/rtdip_sdk/pipelines/deploy/test_databricks_deploy.py +++ b/tests/sdk/python/rtdip_sdk/pipelines/deploy/test_databricks_deploy.py @@ -69,6 +69,7 @@ def test_pipeline_job_deploy(mocker: MockerFixture): mocker.patch("src.sdk.python.rtdip_sdk.pipelines.deploy.databricks.DatabricksSDKDeploy._load_module", return_value=DummyModule()) mocker.patch("src.sdk.python.rtdip_sdk.pipelines.deploy.databricks.DatabricksSDKDeploy._convert_file_to_binary", return_value=None) + mocker.patch("databricks.sdk.mixins.workspace.WorkspaceExt.mkdirs", return_value=None) mocker.patch("databricks.sdk.mixins.workspace.WorkspaceExt.upload", return_value=None) libraries = Libraries(pypi_libraries=[PyPiLibrary(name="rtdip-sdk", version=default_version)], maven_libraries=[MavenLibrary(group_id="rtdip", artifact_id="rtdip-sdk", version=default_version)], python_wheel_libraries=[PythonWheelLibrary(path="test_wheel.whl")]) mocker.patch("src.sdk.python.rtdip_sdk.pipelines.utilities.pipeline_components.PipelineComponentsGetUtility.execute", return_value=(libraries, {"config": "test_config"})) @@ -114,6 +115,7 @@ def test_pipeline_job_deploy_fails(mocker: MockerFixture): mocker.patch("src.sdk.python.rtdip_sdk.pipelines.deploy.databricks.DatabricksSDKDeploy._load_module", return_value=DummyModule()) mocker.patch("src.sdk.python.rtdip_sdk.pipelines.deploy.databricks.DatabricksSDKDeploy._convert_file_to_binary", return_value=None) + mocker.patch("databricks.sdk.mixins.workspace.WorkspaceExt.mkdirs", return_value=None) mocker.patch("databricks.sdk.mixins.workspace.WorkspaceExt.upload", return_value=None) libraries = Libraries(pypi_libraries=[PyPiLibrary(name="rtdip-sdk", version=default_version)], maven_libraries=[MavenLibrary(group_id="rtdip", artifact_id="rtdip-sdk", version=default_version)], python_wheel_libraries=[PythonWheelLibrary(path="test_wheel.whl")]) mocker.patch("src.sdk.python.rtdip_sdk.pipelines.utilities.pipeline_components.PipelineComponentsGetUtility.execute", return_value=(libraries, {"config": "test_config"})) diff --git a/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_eventhub.py b/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_eventhub.py index df5d91438..c1db0f4d5 100644 --- a/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_eventhub.py +++ b/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_eventhub.py @@ -44,7 +44,7 @@ def test_spark_eventhub_write_batch(spark_session: SparkSession, mocker: MockerF assert actual is None def test_spark_eventhub_write_stream(spark_session: SparkSession, mocker: MockerFixture): - mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(return_value=TestStreamingQueryClass())))))))) + mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(queryName=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(return_value=TestStreamingQueryClass())))))))))))) expected_df = spark_session.createDataFrame([{"id": "1"}]) eventhub_destination = SparkEventhubDestination(expected_df, {}) actual = eventhub_destination.write_stream() @@ -58,7 +58,7 @@ def test_spark_eventhub_write_batch_fails(spark_session: SparkSession, mocker: M eventhub_destination.write_batch() def test_spark_eventhub_write_stream_fails(spark_session: SparkSession, mocker: MockerFixture): - mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(side_effect=Exception)))))))) + mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(queryName=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(side_effect=Exception)))))))))))) expected_df = spark_session.createDataFrame([{"id": "1"}]) eventhub_destination = SparkEventhubDestination(expected_df, {}) with pytest.raises(Exception): diff --git a/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_kafka.py b/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_kafka.py index f13902365..58ea5d069 100644 --- a/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_kafka.py +++ b/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_kafka.py @@ -44,7 +44,7 @@ def test_spark_kafka_write_batch(spark_session: SparkSession, mocker: MockerFixt assert actual is None def test_spark_kafka_write_stream(spark_session: SparkSession, mocker: MockerFixture): - mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(return_value=TestStreamingQueryClass())))))))) + mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(queryName=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(return_value=TestStreamingQueryClass())))))))))))) expected_df = spark_session.createDataFrame([{"id": "1"}]) kafka_destination = SparkKafkaDestination(expected_df, {}) actual = kafka_destination.write_stream() @@ -58,7 +58,7 @@ def test_spark_kafka_write_batch_fails(spark_session: SparkSession, mocker: Mock kafka_destination.write_batch() def test_spark_kafka_write_stream_fails(spark_session: SparkSession, mocker: MockerFixture): - mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(side_effect=Exception)))))))) + mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(queryName=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(side_effect=Exception)))))))))))) expected_df = spark_session.createDataFrame([{"id": "1"}]) kafka_destination = SparkKafkaDestination(expected_df, {}) with pytest.raises(Exception): diff --git a/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_kinesis.py b/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_kinesis.py index 1497f11e6..9667d39f3 100644 --- a/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_kinesis.py +++ b/tests/sdk/python/rtdip_sdk/pipelines/destinations/spark/test_kinesis.py @@ -47,21 +47,21 @@ def test_spark_kinesis_write_batch(spark_session: SparkSession, mocker: MockerFi assert actual is None def test_spark_kinesis_write_stream(spark_session: SparkSession, mocker: MockerFixture): - mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(outputMode=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(return_value=TestStreamingQueryClass())))))))))))) + mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(outputMode=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(queryName=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(return_value=TestStreamingQueryClass())))))))))))))) expected_df = spark_session.createDataFrame([{"id": "1"}]) kinesis_destination = SparkKinesisDestination(expected_df, {}, "update", "10 seconds") actual = kinesis_destination.write_stream() assert actual is None def test_spark_kinesis_write_batch_fails(spark_session: SparkSession, mocker: MockerFixture): - mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(outputMode=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(side_effect=Exception)))))))))))) + mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(outputMode=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(queryName=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(side_effect=Exception)))))))))))))) expected_df = spark_session.createDataFrame([{"id": "1"}]) kinesis_destination = SparkKinesisDestination(expected_df, {}, "update", "10 seconds") with pytest.raises(Exception): kinesis_destination.write_batch() def test_spark_kinesis_write_stream_fails(spark_session: SparkSession, mocker: MockerFixture): - mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(outputMode=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(side_effect=Exception)))))))))))) + mocker.patch("pyspark.sql.DataFrame.writeStream", new_callable=mocker.Mock(return_value=mocker.Mock(trigger=mocker.Mock(return_value=mocker.Mock(format=mocker.Mock(return_value=mocker.Mock(outputMode=mocker.Mock(return_value=mocker.Mock(options=mocker.Mock(return_value=mocker.Mock(queryName=mocker.Mock(return_value=mocker.Mock(start=mocker.Mock(return_value=Exception)))))))))))))) expected_df = spark_session.createDataFrame([{"id": "1"}]) kinesis_destination = SparkKinesisDestination(expected_df, {}, "update", "10 seconds") with pytest.raises(Exception): diff --git a/tests/sdk/python/rtdip_sdk/pipelines/sources/spark/test_autoloader.py b/tests/sdk/python/rtdip_sdk/pipelines/sources/spark/test_autoloader.py index 936744a9a..dd21ac44e 100644 --- a/tests/sdk/python/rtdip_sdk/pipelines/sources/spark/test_autoloader.py +++ b/tests/sdk/python/rtdip_sdk/pipelines/sources/spark/test_autoloader.py @@ -39,7 +39,7 @@ def test_databricks_autoloader_read_batch(spark_session: SparkSession): with pytest.raises(NotImplementedError) as excinfo: autoloader_source = DataBricksAutoLoaderSource(spark_session, {}, path, "parquet") autoloader_source.read_batch() - assert str(excinfo.value) == 'Auto Loader only supports streaming reads. To perform a batch read, use the read_stream method and specify Trigger on the write_stream as `availableNow=True`' + assert str(excinfo.value) == 'Auto Loader only supports streaming reads. To perform a batch read, use the read_stream method and specify Trigger on the write_stream as `availableNow`' def test_databricks_autoloader_read_stream(spark_session: SparkSession, mocker: MockerFixture): autoloader_source = DataBricksAutoLoaderSource(spark_session, {}, path, "parquet")