Thanks to visit codestin.com
Credit goes to github.com

Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
71 changes: 69 additions & 2 deletions src/sdk/python/rtdip_sdk/pipelines/destinations/spark/eventhub.py
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,8 @@
import time
from pyspark.sql import DataFrame
from py4j.protocol import Py4JJavaError
from pyspark.sql.functions import col, struct, to_json
from pyspark.sql.types import StringType, BinaryType

from ..interfaces import DestinationInterface
from ..._pipeline_utils.models import Libraries, SystemType
Expand Down Expand Up @@ -79,12 +81,60 @@ def pre_write_validation(self):
def post_write_validation(self):
return True

def prepare_columns(self):
if "body" in self.data.columns:
if self.data.schema["body"].dataType not in [StringType(), BinaryType()]:
try:
self.data.withColumn("body", col("body").cast(StringType()))
except:
raise ValueError("'body' column must be of string or binary type")
else:
self.data = self.data.withColumn(
"body",
to_json(
struct(
[
col(column).alias(column)
for column in self.data.columns
if column not in ["partitionId", "partitionKey"]
]
)
),
)
for column in self.data.schema:
if (
column.name in ["partitionId", "partitionKey"]
and column.dataType != StringType()
):
try:
self.data = self.data.withColumn(
column.name, col(column.name).cast(StringType())
)
except:
raise ValueError(f"Column {column.name} must be of string type")
return self.data.select(
[
column
for column in self.data.columns
if column in ["partitionId", "partitionKey", "body"]
]
)

def write_batch(self):
"""
Writes batch data to Eventhubs.
"""
eventhub_connection_string = "eventhubs.connectionString"
try:
return self.data.write.format("eventhubs").options(**self.options).save()
if eventhub_connection_string in self.options:
sc = self.spark.sparkContext
self.options[
eventhub_connection_string
] = sc._jvm.org.apache.spark.eventhubs.EventHubsUtils.encrypt(
self.options[eventhub_connection_string]
)
df = self.prepare_columns()
return df.write.format("eventhubs").options(**self.options).save()

except Py4JJavaError as e:
logging.exception(e.errmsg)
Expand All @@ -97,14 +147,31 @@ def write_stream(self):
"""
Writes steaming data to Eventhubs.
"""
eventhub_connection_string = "eventhubs.connectionString"
try:
TRIGGER_OPTION = (
{"availableNow": True}
if self.trigger == "availableNow"
else {"processingTime": self.trigger}
)
if eventhub_connection_string in self.options:
sc = self.spark.sparkContext
self.options[
eventhub_connection_string
] = sc._jvm.org.apache.spark.eventhubs.EventHubsUtils.encrypt(
self.options[eventhub_connection_string]
)
df = self.prepare_columns()
df = self.data.select(
[
column
for column in self.data.columns
if column in ["partitionId", "partitionKey", "body"]
]
)

query = (
self.data.writeStream.trigger(**TRIGGER_OPTION)
df.writeStream.trigger(**TRIGGER_OPTION)
.format("eventhubs")
.options(**self.options)
.queryName(self.query_name)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -29,7 +29,6 @@
from pyspark.sql import Window
from datetime import datetime
import pytz
import uuid

from ..interfaces import TransformerInterface
from ..._pipeline_utils.models import Libraries, SystemType
Expand All @@ -42,7 +41,7 @@ class PCDMToHoneywellAPMTransformer(TransformerInterface):
Args:
data (Dataframe): Spark Dataframe in PCDM format
quality (str): Value for quality inside HistorySamples
history_samples_per_message (int): The number of HistorySamples for each row in the DataFrame
history_samples_per_message (int): The number of HistorySamples for each row in the DataFrame (Batch Only)

"""

Expand Down Expand Up @@ -88,63 +87,74 @@ def transform(self) -> DataFrame:
Returns:
DataFrame: A dataframe with with rows in Honeywell APM format
"""
pcdm_df = self.data.withColumn("counter", monotonically_increasing_id())
w = Window.orderBy("counter")
indexed_pcdm_df = (
pcdm_df.withColumn(
"index",
floor((row_number().over(w) - 0.01) / self.history_samples_per_message),
)
.withColumn(
"HistorySamples",
struct(
col("TagName").alias("ItemName"),
lit(self.quality).alias("Quality"),
col("EventTime").alias("Time"),
col("Value").alias("Value"),
).alias("HistorySamples"),
if self.data.isStreaming and self.history_samples_per_message > 1:
pcdm_df = self.data.withColumn("counter", monotonically_increasing_id())
w = Window.orderBy("counter")
cleaned_pcdm_df = (
pcdm_df.withColumn(
"index",
floor(
(row_number().over(w) - 0.01) / self.history_samples_per_message
),
)
.withColumn(
"HistorySamples",
struct(
col("TagName").alias("ItemName"),
lit(self.quality).alias("Quality"),
col("EventTime").alias("Time"),
col("Value").alias("Value"),
).alias("HistorySamples"),
)
.groupBy("index")
.agg(collect_list("HistorySamples").alias("HistorySamples"))
.withColumn("guid", expr("uuid()"))
.withColumn(
"value",
struct(
col("guid").alias("SystemGuid"), col("HistorySamples")
).alias("value"),
)
)
.groupBy("index")
.agg(collect_list("HistorySamples").alias("HistorySamples"))
.withColumn("guid", expr("uuid()"))
.withColumn(
else:
cleaned_pcdm_df = self.data.withColumn("guid", expr("uuid()")).withColumn(
"value",
struct(col("guid").alias("SystemGuid"), col("HistorySamples")).alias(
"value"
struct(
col("guid").alias("SystemGuid"),
struct(
col("TagName").alias("ItemName"),
lit(self.quality).alias("Quality"),
col("EventTime").alias("Time"),
col("Value").alias("Value"),
).alias("HistorySamples"),
),
)
)

df = indexed_pcdm_df.withColumn(
df = cleaned_pcdm_df.withColumn(
"CloudPlatformEvent",
create_map(
lit("CloudPlatformEvent"),
struct(
lit(datetime.now(tz=pytz.UTC)).alias("CreatedTime"),
lit(expr("uuid()")).alias("Id"),
col("guid").alias("CreatorId"),
lit("CloudPlatformSystem").alias("CreatorType"),
lit(None).alias("GeneratorId"),
lit("CloudPlatformTenant").alias("GeneratorType"),
col("guid").alias("TargetId"),
lit("CloudPlatformTenant").alias("TargetType"),
lit(None).alias("TargetContext"),
struct(
lit(datetime.now(tz=pytz.UTC)).alias("CreatedTime"),
lit(expr("uuid()")).alias("Id"),
col("guid").alias("CreatorId"),
lit("CloudPlatformSystem").alias("CreatorType"),
lit(None).alias("GeneratorId"),
lit("CloudPlatformTenant").alias("GeneratorType"),
col("guid").alias("TargetId"),
lit("CloudPlatformTenant").alias("TargetType"),
lit(None).alias("TargetContext"),
lit("TextualBody").alias("type"),
to_json(col("value")).alias("value"),
lit("application/json").alias("format"),
).alias("Body"),
array(
struct(
lit("TextualBody").alias("type"),
to_json(col("value")).alias("value"),
lit("application/json").alias("format"),
).alias("Body"),
array(
struct(
lit("SystemType").alias("Key"),
lit("apm-system").alias("Value"),
),
struct(
lit("SystemGuid").alias("Key"), col("guid").alias("Value")
),
).alias("BodyProperties"),
lit("DataChange.Update").alias("EventType"),
),
lit("SystemType").alias("Key"),
lit("apm-system").alias("Value"),
),
struct(lit("SystemGuid").alias("Key"), col("guid").alias("Value")),
).alias("BodyProperties"),
lit("DataChange.Update").alias("EventType"),
),
).withColumn("AnnotationStreamIds", lit("self.AnnotationStreamIds"))

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,14 @@
MavenLibrary,
)
from pyspark.sql import SparkSession, DataFrame
from pyspark.sql.streaming import StreamingQuery
from pyspark.sql.types import (
StructType,
StructField,
StringType,
TimestampType,
LongType,
)
from datetime import datetime


class TestStreamingQueryClass:
Expand All @@ -51,6 +58,49 @@ def test_spark_eventhub_write_setup():
assert eventhub_destination.post_write_validation()


def test_prepare_columns(spark_session: SparkSession):
pcdm_schema = StructType(
[
StructField("TagName", StringType(), True),
StructField("EventTime", TimestampType(), True),
StructField("Status", StringType(), False),
StructField("Value", StringType(), True),
StructField("ValueType", StringType(), False),
StructField("ChangeType", StringType(), False),
StructField("partitionId", LongType(), False),
]
)

pcdm_data = [
{
"TagName": "test.item1",
"EventTime": datetime.fromisoformat("2023-07-31T06:53:00+00:00"),
"Status": "Good",
"Value": 5.0,
"ValueType": "float",
"ChangeType": "insert",
"partitionId": 134343345,
},
{
"TagName": "Test_item2",
"EventTime": datetime.fromisoformat("2023-07-31T06:54:00+00:00"),
"Status": "Good",
"Value": 1,
"ValueType": "float",
"ChangeType": "insert",
"partitionId": 134343345,
},
]
pcdm_df: DataFrame = spark_session.createDataFrame(
schema=pcdm_schema, data=pcdm_data
)
eventhub_destination = SparkEventhubDestination(pcdm_df, {})
prepared_df = eventhub_destination.prepare_columns()
assert len(prepared_df.schema) == 2
assert prepared_df.schema["body"].dataType == StringType()
assert prepared_df.schema["partitionId"].dataType == StringType()


def test_spark_eventhub_write_batch(spark_session: SparkSession, mocker: MockerFixture):
mocker.patch(
"pyspark.sql.DataFrame.write",
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,9 +30,8 @@
from pyspark.sql import SparkSession, DataFrame
from pytest_mock import MockerFixture
from pyspark.sql.types import StructType, StructField, StringType, TimestampType
from pyspark.sql.functions import regexp_replace
from datetime import datetime
import json
import uuid


def test_pcdm_to_honeywell_apm(spark_session: SparkSession, mocker: MockerFixture):
Expand Down Expand Up @@ -65,50 +64,23 @@ def test_pcdm_to_honeywell_apm(spark_session: SparkSession, mocker: MockerFixtur
"ChangeType": "insert",
},
]

pcdm_df: DataFrame = spark_session.createDataFrame(
schema=pcdm_schema, data=pcdm_data
)
honeywell_json_data = {
"CloudPlatformEvent": {
"CreatedTime": "2023-08-10T06:53:00+00:00",
"Id": "2b2a64f6-bfee-49f5-9d1b-04df844e80be",
"CreatorId": "065a7343-a3b5-4ecd-9bac-19cdff5cf048",
"CreatorType": "CloudPlatformSystem",
"GeneratorId": None,
"GeneratorType": "CloudPlatformTenant",
"TargetId": "065a7343-a3b5-4ecd-9bac-19cdff5cf048",
"TargetType": "CloudPlatformTenant",
"TargetContext": None,
"Body": {
"type": "TextualBody",
"value": '{"SystemGuid":"065a7343-a3b5-4ecd-9bac-19cdff5cf048","HistorySamples":[{"ItemName":"test.item1","Quality":"Good","Time":"2023-07-31T06:53:00+00:00","Value":5},{"ItemName":"Test_item2","Quality":"Good","Time":"2023-07-31T06:54:00+00:00","Value":1}]}',
"format": "application/json",
},
"BodyProperties": [
{"Key": "SystemType", "Value": "apm-system"},
{"Key": "SystemGuid", "Value": "065a7343-a3b5-4ecd-9bac-19cdff5cf048"},
],
"EventType": "DataChange.Update",
},
"AnnotationStreamIds": ",",
}
expected_df = spark_session.createDataFrame([honeywell_json_data])
PCDM_to_honeywell_eventhub_json_transformer = PCDMToHoneywellAPMTransformer(
data=pcdm_df, history_samples_per_message=3
)

actual_df = PCDM_to_honeywell_eventhub_json_transformer.transform()
dict = actual_df.collect()[0]["CloudPlatformEvent"]

assert len(dict) == 1
dict = actual_df.collect()[0]
assert isinstance(uuid.UUID(dict["CloudPlatformEvent"]["CreatorId"]), uuid.UUID)
assert (
PCDM_to_honeywell_eventhub_json_transformer.system_type() == SystemType.PYSPARK
)
assert isinstance(
PCDM_to_honeywell_eventhub_json_transformer.libraries(), Libraries
)
assert len(dict) == 1
assert len(dict) == 2
assert len(dict["CloudPlatformEvent"]) == 12
assert len(dict["CloudPlatformEvent"]["Body"]) == 3
assert len(dict["CloudPlatformEvent"]["BodyProperties"]) == 2
Expand Down