Skip to content

Commit

Permalink
[dagster-embedded-elt] Add relation identifier metadata to sling asse…
Browse files Browse the repository at this point in the history
…ts (dagster-io#25016)

## Summary & Motivation

Pull `target` from a sling replication config and use it as a
destination name create a relation identifier when a sling asset is
materialized.

<img width="1367" alt="Screenshot 2024-10-02 at 5 51 22 PM"
src="https://github.com/user-attachments/assets/110266c0-b1d6-4a3c-8425-9fa642cc575f">

## How I Tested These Changes

BK with new test + tested with a test pipeline.

## Changelog

[dagster-embedded-elt] relation identifier metadata is now attached to
sling assets.

- [x] `NEW` _(added new feature or capability)_
- [ ] `BUGFIX` _(fixed a bug)_
- [ ] `DOCS` _(added or updated documentation)_
  • Loading branch information
maximearmstrong authored Oct 3, 2024
1 parent 64c894e commit 4136f3d
Show file tree
Hide file tree
Showing 2 changed files with 48 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
get_dagster_logger,
)
from dagster._annotations import public
from dagster._core.definitions.metadata import TableMetadataSet
from dagster._utils.env import environ
from pydantic import Field

Expand Down Expand Up @@ -358,7 +359,10 @@ def _replicate(
context_streams = self._get_replication_streams_for_context(context)
if context_streams:
replication_config.update({"streams": context_streams})
stream_definition = get_streams_from_replication(replication_config)
stream_definitions = get_streams_from_replication(replication_config)

# extract the destination name from the replication config
destination_name = replication_config.get("target")

with self._setup_config():
uid = uuid.uuid4()
Expand Down Expand Up @@ -394,12 +398,21 @@ def _replicate(

# TODO: In the future, it'd be nice to yield these materializations as they come in
# rather than waiting until the end of the replication
for stream in stream_definition:
for stream in stream_definitions:
asset_key = dagster_sling_translator.get_asset_key(stream)

object_key = stream.get("config", {}).get("object")
destination_stream_name = object_key or stream["name"]
relation_identifier = None
if destination_name and destination_stream_name:
relation_identifier = ".".join([destination_name, destination_stream_name])

metadata = {
"elapsed_time": end_time - start_time,
"stream_name": stream["name"],
**TableMetadataSet(
relation_identifier=relation_identifier,
),
}

if context.has_assets_def:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@
file_relative_path,
)
from dagster._core.definitions.materialize import materialize
from dagster._core.definitions.metadata.metadata_value import TextMetadataValue
from dagster._core.definitions.tags import build_kind_tag
from dagster_embedded_elt.sling import SlingReplicationParam, sling_assets
from dagster_embedded_elt.sling.dagster_sling_translator import DagsterSlingTranslator
Expand Down Expand Up @@ -466,3 +467,35 @@ def my_sling_assets(
assert found_asset_keys == {
AssetKey(["target", "main", "orders"]),
}


def test_relation_identifier(
csv_to_sqlite_dataworks_replication: SlingReplicationParam,
path_to_temp_sqlite_db: str,
):
@sling_assets(replication_config=csv_to_sqlite_dataworks_replication)
def my_sling_assets(context: AssetExecutionContext, sling: SlingResource):
yield from sling.replicate(context=context)

sling_resource = SlingResource(
connections=[
SlingConnectionResource(type="file", name="SLING_FILE"),
SlingConnectionResource(
type="sqlite",
name="SLING_SQLITE",
connection_string=f"sqlite://{path_to_temp_sqlite_db}",
),
]
)
res = materialize(
[my_sling_assets],
resources={"sling": sling_resource},
selection=[AssetKey(["target", "main", "orders"])],
)

assert res.success
asset_materializations = res.get_asset_materialization_events()
assert len(asset_materializations) == 1
assert asset_materializations[0].materialization.metadata[
"dagster/relation_identifier"
] == TextMetadataValue(text="SLING_SQLITE.main.orders")

0 comments on commit 4136f3d

Please sign in to comment.