diff --git a/docs/content/concepts/metadata-tags/asset-metadata.mdx b/docs/content/concepts/metadata-tags/asset-metadata.mdx
index 4929fdc552566..6bacb91932e90 100644
--- a/docs/content/concepts/metadata-tags/asset-metadata.mdx
+++ b/docs/content/concepts/metadata-tags/asset-metadata.mdx
@@ -473,7 +473,7 @@ The `dagster` prefix indicates that the Dagster package takes responsibility for
- dagster/relation_identifier
+ dagster/table_name
|
For an asset that's a table, the lineage of column inputs to column outputs for the table. Refer to the [Table and column metadata](#table-schema) section for details. |
| `dagster/row_count` | **Type:** `int`
For an asset that's a table, the number of rows in the table. Refer to the Table metadata documentation for details. |
| `dagster/partition_row_count` | **Type:** `int`
For a partition of an asset that's a table, the number of rows in the partition. |
-| `dagster/relation_identifier` | **Type:** `str`
A unique identifier for the table/view, typically fully qualified. For example, my_database.my_schema.my_table |
+| `dagster/table_name` | **Type:** `str`
A unique identifier for the table/view, typically fully qualified. For example, my_database.my_schema.my_table |
| `dagster/code_references` | **Type:** [`CodeReferencesMetadataValue`](/todo)
A list of code references for the asset, such as file locations or references to GitHub URLs. Refer to the [Linking assets with their source code](#source-code) section for details. Should only be provided in definition-level metadata, not materialization metadata. |
## Table and column metadata \{#table-column}
diff --git a/js_modules/dagster-ui/packages/ui-core/src/assets/AssetNodeOverview.tsx b/js_modules/dagster-ui/packages/ui-core/src/assets/AssetNodeOverview.tsx
index 11e4aa61c3b6c..73c8a3a5edcbe 100644
--- a/js_modules/dagster-ui/packages/ui-core/src/assets/AssetNodeOverview.tsx
+++ b/js_modules/dagster-ui/packages/ui-core/src/assets/AssetNodeOverview.tsx
@@ -67,7 +67,7 @@ import {
TableSchema,
TableSchemaAssetContext,
isCanonicalCodeSourceEntry,
- isCanonicalRelationIdentifierEntry,
+ isCanonicalTableNameEntry,
isCanonicalUriEntry,
} from '../metadata/TableSchema';
import {RepositoryLink} from '../nav/RepositoryLink';
@@ -294,9 +294,7 @@ export const AssetNodeOverview = ({
const nonSystemTags = filteredTags?.filter((tag) => !isSystemTag(tag));
const systemTags = filteredTags?.filter(isSystemTag);
- const relationIdentifierMetadata = assetNode?.metadataEntries?.find(
- isCanonicalRelationIdentifierEntry,
- );
+ const tableNameMetadata = assetNode?.metadataEntries?.find(isCanonicalTableNameEntry);
const uriMetadata = assetNode?.metadataEntries?.find(isCanonicalUriEntry);
const codeSource = assetNode?.metadataEntries?.find((m) => isCanonicalCodeSourceEntry(m)) as
@@ -370,12 +368,12 @@ export const AssetNodeOverview = ({
))}
- {(relationIdentifierMetadata || uriMetadata || storageKindTag) && (
+ {(tableNameMetadata || uriMetadata || storageKindTag) && (
- {relationIdentifierMetadata && (
+ {tableNameMetadata && (
-
-
+
+
)}
{uriMetadata && (
diff --git a/js_modules/dagster-ui/packages/ui-core/src/metadata/TableSchema.tsx b/js_modules/dagster-ui/packages/ui-core/src/metadata/TableSchema.tsx
index 666e6945c7dcc..1fbb573223445 100644
--- a/js_modules/dagster-ui/packages/ui-core/src/metadata/TableSchema.tsx
+++ b/js_modules/dagster-ui/packages/ui-core/src/metadata/TableSchema.tsx
@@ -52,9 +52,8 @@ export const isCanonicalCodeSourceEntry = (
): m is CodeReferencesMetadataEntry =>
m && m.__typename === 'CodeReferencesMetadataEntry' && m.label === 'dagster/code_references';
-export const isCanonicalRelationIdentifierEntry = (
- m: MetadataEntryLabelOnly,
-): m is TextMetadataEntry => m && m.label === 'dagster/relation_identifier';
+export const isCanonicalTableNameEntry = (m: MetadataEntryLabelOnly): m is TextMetadataEntry =>
+ m && (m.label === 'dagster/relation_identifier' || m.label === 'dagster/table_name');
export const isCanonicalUriEntry = (
m: MetadataEntryLabelOnly,
diff --git a/python_modules/dagster/dagster/_core/definitions/metadata/metadata_set.py b/python_modules/dagster/dagster/_core/definitions/metadata/metadata_set.py
index e750c88e67038..244632c287f70 100644
--- a/python_modules/dagster/dagster/_core/definitions/metadata/metadata_set.py
+++ b/python_modules/dagster/dagster/_core/definitions/metadata/metadata_set.py
@@ -165,7 +165,7 @@ class TableMetadataSet(NamespacedMetadataSet):
outputs for the table.
row_count (Optional[int]): The number of rows in the table.
partition_row_count (Optional[int]): The number of rows in the materialized or observed partition.
- relation_identifier (Optional[str]): A unique identifier for the table/view, typically fully qualified.
+ table_name (Optional[str]): A unique identifier for the table/view, typically fully qualified.
For example, `my_database.my_schema.my_table`.
"""
@@ -173,7 +173,7 @@ class TableMetadataSet(NamespacedMetadataSet):
column_lineage: Optional[TableColumnLineage] = None
row_count: Optional[int] = None
partition_row_count: Optional[int] = None
- relation_identifier: Optional[str] = None
+ table_name: Optional[str] = None
@classmethod
def namespace(cls) -> str:
diff --git a/python_modules/dagster/dagster/_core/storage/db_io_manager.py b/python_modules/dagster/dagster/_core/storage/db_io_manager.py
index 39ae22947446d..1479a04bf9093 100644
--- a/python_modules/dagster/dagster/_core/storage/db_io_manager.py
+++ b/python_modules/dagster/dagster/_core/storage/db_io_manager.py
@@ -78,8 +78,8 @@ def delete_table_slice(
def get_select_statement(table_slice: TableSlice) -> str: ...
@staticmethod
- def get_relation_identifier(table_slice: TableSlice) -> Optional[str]:
- """Returns a string which is set as the dagster/relation_identifier metadata value for an
+ def get_table_name(table_slice: TableSlice) -> Optional[str]:
+ """Returns a string which is set as the dagster/table_name metadata value for an
emitted asset. This value should be the fully qualified name of the table, including the
schema and database, if applicable.
"""
@@ -171,11 +171,7 @@ def handle_output(self, context: OutputContext, obj: object) -> None:
# don't fail if it errors because the user has already attached it.
try:
context.add_output_metadata(
- dict(
- TableMetadataSet(
- relation_identifier=self._db_client.get_relation_identifier(table_slice)
- )
- )
+ dict(TableMetadataSet(table_name=self._db_client.get_table_name(table_slice)))
)
except DagsterInvalidMetadata:
pass
diff --git a/python_modules/dagster/dagster_tests/definitions_tests/metadata_tests/test_table_metadata_set.py b/python_modules/dagster/dagster_tests/definitions_tests/metadata_tests/test_table_metadata_set.py
index b9739595b1992..0af1d9618332f 100644
--- a/python_modules/dagster/dagster_tests/definitions_tests/metadata_tests/test_table_metadata_set.py
+++ b/python_modules/dagster/dagster_tests/definitions_tests/metadata_tests/test_table_metadata_set.py
@@ -31,11 +31,11 @@ def test_table_metadata_set() -> None:
assert TableMetadataSet.extract(dict(TableMetadataSet())) == TableMetadataSet()
-def test_relation_identifier() -> None:
- table_metadata = TableMetadataSet(relation_identifier="my_database.my_schema.my_table")
+def test_table_name() -> None:
+ table_metadata = TableMetadataSet(table_name="my_database.my_schema.my_table")
dict_table_metadata = dict(table_metadata)
- assert dict_table_metadata == {"dagster/relation_identifier": "my_database.my_schema.my_table"}
+ assert dict_table_metadata == {"dagster/table_name": "my_database.my_schema.my_table"}
AssetMaterialization(asset_key="a", metadata=dict_table_metadata)
diff --git a/python_modules/dagster/dagster_tests/storage_tests/test_db_io_manager.py b/python_modules/dagster/dagster_tests/storage_tests/test_db_io_manager.py
index 547168064cc12..4880c9836bfed 100644
--- a/python_modules/dagster/dagster_tests/storage_tests/test_db_io_manager.py
+++ b/python_modules/dagster/dagster_tests/storage_tests/test_db_io_manager.py
@@ -25,8 +25,8 @@
}
-def mock_relation_identifier(*args, **kwargs) -> str:
- return "relation_identifier"
+def mock_table_name(*args, **kwargs) -> str:
+ return "table_name"
class IntHandler(DbTypeHandler[int]):
@@ -81,7 +81,7 @@ def test_asset_out():
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
connect=connect_mock,
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = build_db_io_manager(type_handlers=[handler], db_client=db_client)
asset_key = AssetKey(["schema1", "table1"])
@@ -117,7 +117,7 @@ def test_asset_out_columns():
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
connect=connect_mock,
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = build_db_io_manager(type_handlers=[handler], db_client=db_client)
asset_key = AssetKey(["schema1", "table1"])
@@ -159,7 +159,7 @@ def test_asset_out_partitioned():
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
connect=connect_mock,
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = build_db_io_manager(type_handlers=[handler], db_client=db_client)
asset_key = AssetKey(["schema1", "table1"])
@@ -218,7 +218,7 @@ def test_asset_out_static_partitioned():
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
connect=connect_mock,
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = build_db_io_manager(type_handlers=[handler], db_client=db_client)
asset_key = AssetKey(["schema1", "table1"])
@@ -270,7 +270,7 @@ def test_asset_out_multiple_static_partitions():
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
connect=connect_mock,
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = build_db_io_manager(type_handlers=[handler], db_client=db_client)
asset_key = AssetKey(["schema1", "table1"])
@@ -323,7 +323,7 @@ def test_different_output_and_input_types():
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
connect=connect_mock,
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = build_db_io_manager(type_handlers=[int_handler, str_handler], db_client=db_client)
asset_key = AssetKey(["schema1", "table1"])
@@ -361,7 +361,7 @@ def test_non_asset_out():
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
connect=connect_mock,
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = build_db_io_manager(type_handlers=[handler], db_client=db_client)
output_context = build_output_context(
@@ -399,7 +399,7 @@ def test_asset_schema_defaults():
db_client = MagicMock(
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = build_db_io_manager(type_handlers=[handler], db_client=db_client)
@@ -561,7 +561,7 @@ def test_default_load_type():
db_client = MagicMock(
spec=DbClient,
get_select_statement=MagicMock(return_value=""),
- get_relation_identifier=mock_relation_identifier,
+ get_table_name=mock_table_name,
)
manager = DbIOManager(
type_handlers=[handler],
diff --git a/python_modules/libraries/dagster-airbyte/dagster_airbyte/asset_defs.py b/python_modules/libraries/dagster-airbyte/dagster_airbyte/asset_defs.py
index cacf4344d5438..9b64a59c0d44f 100644
--- a/python_modules/libraries/dagster-airbyte/dagster_airbyte/asset_defs.py
+++ b/python_modules/libraries/dagster-airbyte/dagster_airbyte/asset_defs.py
@@ -116,11 +116,11 @@ def _build_airbyte_asset_defn_metadata(
for table in destination_tables:
internal_deps[table] = set(upstream_assets or [])
- relation_identifiers: Dict[str, str] = {}
+ table_names: Dict[str, str] = {}
for table in destination_tables:
if destination_database and destination_schema and table:
- # Use the destination raw table name to create the relation identifier
- relation_identifiers[table] = ".".join(
+ # Use the destination raw table name to create the table name
+ table_names[table] = ".".join(
[
destination_database,
destination_schema,
@@ -129,7 +129,7 @@ def _build_airbyte_asset_defn_metadata(
)
if normalization_tables and normalization_raw_table_names_by_table:
for normalization_table in normalization_tables.get(table, set()):
- relation_identifiers[normalization_table] = ".".join(
+ table_names[normalization_table] = ".".join(
[
destination_database,
destination_schema,
@@ -156,7 +156,7 @@ def _build_airbyte_asset_defn_metadata(
table: {
**TableMetadataSet(
column_schema=schema_by_table_name.get(table),
- relation_identifier=relation_identifiers.get(table),
+ table_name=table_names.get(table),
),
}
for table in tables
@@ -302,15 +302,13 @@ def build_airbyte_assets(
chain([destination_tables], normalization_tables.values() if normalization_tables else [])
)
- relation_identifiers: Dict[str, str] = {}
+ table_names: Dict[str, str] = {}
for table in destination_tables:
if destination_database and destination_schema and table:
- relation_identifiers[table] = ".".join(
- [destination_database, destination_schema, table]
- )
+ table_names[table] = ".".join([destination_database, destination_schema, table])
if normalization_tables:
for normalization_table in normalization_tables.get(table, set()):
- relation_identifiers[normalization_table] = ".".join(
+ table_names[normalization_table] = ".".join(
[
destination_database,
destination_schema,
@@ -328,7 +326,7 @@ def build_airbyte_assets(
{
**TableMetadataSet(
column_schema=schema_by_table_name.get(table),
- relation_identifier=relation_identifiers.get(table),
+ table_name=table_names.get(table),
),
}
),
diff --git a/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_asset_defs.py b/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_asset_defs.py
index 04b2ed8d3ddfa..089d12568b992 100644
--- a/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_asset_defs.py
+++ b/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_asset_defs.py
@@ -310,7 +310,7 @@ def downstream_of_ab():
assert downstream_of_ab.op.ins["some_prefix_bar"].dagster_type.is_nothing
-def test_built_airbyte_asset_relation_identifier():
+def test_built_airbyte_asset_table_name():
destination_tables = ["foo", "bar"]
ab_assets = build_airbyte_assets(
@@ -322,7 +322,7 @@ def test_built_airbyte_asset_relation_identifier():
# Check relation identifier metadata is added correctly to asset def
assets_def = ab_assets[0]
for metadata in assets_def.metadata_by_key.values():
- assert metadata.get("dagster/relation_identifier") is None
+ assert metadata.get("dagster/table_name") is None
ab_assets = build_airbyte_assets(
"12345",
@@ -331,15 +331,15 @@ def test_built_airbyte_asset_relation_identifier():
destination_schema="test_schema",
)
- relation_identifiers = {"test_database.test_schema.foo", "test_database.test_schema.bar"}
+ table_names = {"test_database.test_schema.foo", "test_database.test_schema.bar"}
# Check relation identifier metadata is added correctly to asset def
assets_def = ab_assets[0]
for key, metadata in assets_def.metadata_by_key.items():
# Extract the table name from the asset key
table_name = key.path[-1]
- assert metadata["dagster/relation_identifier"] in relation_identifiers
- assert table_name in metadata["dagster/relation_identifier"]
+ assert metadata["dagster/table_name"] in table_names
+ assert table_name in metadata["dagster/table_name"]
ab_assets = build_airbyte_assets(
"12345",
@@ -349,12 +349,12 @@ def test_built_airbyte_asset_relation_identifier():
normalization_tables={"foo": {"baz"}},
)
- relation_identifiers.add("test_database.test_schema.foo.baz")
+ table_names.add("test_database.test_schema.foo.baz")
# Check relation identifier metadata is added correctly to asset def
assets_def = ab_assets[0]
for key, metadata in assets_def.metadata_by_key.items():
# Extract the table name from the asset key
table_name = key.path[-1]
- assert metadata["dagster/relation_identifier"] in relation_identifiers
- assert table_name in metadata["dagster/relation_identifier"]
+ assert metadata["dagster/table_name"] in table_names
+ assert table_name in metadata["dagster/table_name"]
diff --git a/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_load_from_instance.py b/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_load_from_instance.py
index 58bea77c3974b..e64f5edfd8986 100644
--- a/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_load_from_instance.py
+++ b/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_load_from_instance.py
@@ -234,7 +234,7 @@ def downstream_asset(dagster_tags):
for key, metadata in assets_def.metadata_by_key.items()
)
- relation_identifiers = {
+ table_names = {
"test_database.test_schema.releases",
"test_database.test_schema.tags",
"test_database.test_schema.teams",
@@ -262,10 +262,10 @@ def downstream_asset(dagster_tags):
.replace("_test", "")
.split("_")[-1]
)
- assert metadata["dagster/relation_identifier"] in relation_identifiers
- assert table_name in metadata["dagster/relation_identifier"]
+ assert metadata["dagster/table_name"] in table_names
+ assert table_name in metadata["dagster/table_name"]
else:
- assert not metadata.get("dagster/relation_identifier")
+ assert not metadata.get("dagster/table_name")
assert assets_def.keys == {AssetKey(t) for t in tables}
assert all(
diff --git a/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_load_from_project.py b/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_load_from_project.py
index 31af15e5590d1..58df98d541514 100644
--- a/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_load_from_project.py
+++ b/python_modules/libraries/dagster-airbyte/dagster_airbyte_tests/test_load_from_project.py
@@ -106,7 +106,7 @@ def test_load_from_project(
# Check metadata is added correctly to asset def
assets_def = ab_assets[0]
- relation_identifiers = {
+ table_names = {
"AIRBYTE.BEN_DEMO.releases",
"AIRBYTE.BEN_DEMO.tags",
"AIRBYTE.BEN_DEMO.teams",
@@ -133,8 +133,8 @@ def test_load_from_project(
.replace("_test", "")
.split("_")[-1]
)
- assert metadata["dagster/relation_identifier"] in relation_identifiers
- assert table_name in metadata["dagster/relation_identifier"]
+ assert metadata["dagster/table_name"] in table_names
+ assert table_name in metadata["dagster/table_name"]
assert assets_def.keys == {AssetKey(t) for t in tables}
assert all(
diff --git a/python_modules/libraries/dagster-dbt/dagster_dbt/asset_utils.py b/python_modules/libraries/dagster-dbt/dagster_dbt/asset_utils.py
index 763b22874e18c..c169f84eee4fd 100644
--- a/python_modules/libraries/dagster-dbt/dagster_dbt/asset_utils.py
+++ b/python_modules/libraries/dagster-dbt/dagster_dbt/asset_utils.py
@@ -469,7 +469,7 @@ def default_metadata_from_dbt_resource_props(
**DbtMetadataSet(materialization_type=materialization_type),
**TableMetadataSet(
column_schema=column_schema,
- relation_identifier=relation_name,
+ table_name=relation_name,
),
}
diff --git a/python_modules/libraries/dagster-dbt/dagster_dbt_tests/core/test_def_metadata.py b/python_modules/libraries/dagster-dbt/dagster_dbt_tests/core/test_def_metadata.py
index f401c3ab579a8..6ab9340c3760e 100644
--- a/python_modules/libraries/dagster-dbt/dagster_dbt_tests/core/test_def_metadata.py
+++ b/python_modules/libraries/dagster-dbt/dagster_dbt_tests/core/test_def_metadata.py
@@ -44,21 +44,20 @@ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
}
assert all(
- storage_address_meta.relation_identifier
- for storage_address_meta in storage_address_metas.values()
+ storage_address_meta.table_name for storage_address_meta in storage_address_metas.values()
)
jaffle_shop_duckdb_dbfile_name = os.getenv("DAGSTER_DBT_PYTEST_XDIST_DUCKDB_DBFILE_NAME")
# spot check a few storage addresses
assert (
- storage_address_metas["customers"].relation_identifier
+ storage_address_metas["customers"].table_name
== f"{jaffle_shop_duckdb_dbfile_name}.dev.customers"
)
assert (
- storage_address_metas["raw_customers"].relation_identifier
+ storage_address_metas["raw_customers"].table_name
== f"{jaffle_shop_duckdb_dbfile_name}.dev.raw_customers"
)
assert (
- storage_address_metas["stg_orders"].relation_identifier
+ storage_address_metas["stg_orders"].table_name
== f"{jaffle_shop_duckdb_dbfile_name}.dev.stg_orders"
)
@@ -76,18 +75,17 @@ def my_dbt_assets(context: AssetExecutionContext, dbt: DbtCliResource):
}
assert all(
- storage_address_meta.relation_identifier
- for storage_address_meta in storage_address_metas.values()
+ storage_address_meta.table_name for storage_address_meta in storage_address_metas.values()
)
jaffle_shop_duckdb_dbfile_name = os.getenv("DAGSTER_DBT_PYTEST_XDIST_DUCKDB_DBFILE_NAME")
# test that we can have tables with dots in their names, from
# user-defined aliases
assert (
- storage_address_metas["customers"].relation_identifier
+ storage_address_metas["customers"].table_name
== f"{jaffle_shop_duckdb_dbfile_name}.main.dagster.customers"
)
assert (
- storage_address_metas["orders"].relation_identifier
+ storage_address_metas["orders"].table_name
== f"{jaffle_shop_duckdb_dbfile_name}.main.dagster.orders"
)
diff --git a/python_modules/libraries/dagster-duckdb-pandas/dagster_duckdb_pandas_tests/test_type_handler.py b/python_modules/libraries/dagster-duckdb-pandas/dagster_duckdb_pandas_tests/test_type_handler.py
index afb7be5766b40..55202d98470bd 100644
--- a/python_modules/libraries/dagster-duckdb-pandas/dagster_duckdb_pandas_tests/test_type_handler.py
+++ b/python_modules/libraries/dagster-duckdb-pandas/dagster_duckdb_pandas_tests/test_type_handler.py
@@ -125,7 +125,7 @@ def my_pandas_df() -> pd.DataFrame:
assert len(mats) == 1
mat = mats[0]
- assert mat.materialization.metadata["dagster/relation_identifier"] == MetadataValue.text(
+ assert mat.materialization.metadata["dagster/table_name"] == MetadataValue.text(
f"{db_file}.custom_schema.my_pandas_df"
)
diff --git a/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt/dlt/resource.py b/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt/dlt/resource.py
index ce68a02f7fe60..a72ac1714687b 100644
--- a/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt/dlt/resource.py
+++ b/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt/dlt/resource.py
@@ -151,9 +151,9 @@ def extract_resource_metadata(
break
destination_name: Optional[str] = base_metadata.get("destination_name")
- relation_identifier = None
+ table_name = None
if destination_name and schema:
- relation_identifier = ".".join([destination_name, schema, normalized_table_name])
+ table_name = ".".join([destination_name, schema, normalized_table_name])
child_table_names = [
name
@@ -171,7 +171,7 @@ def extract_resource_metadata(
**base_metadata,
**TableMetadataSet(
column_schema=table_schema,
- relation_identifier=relation_identifier,
+ table_name=table_name,
),
}
diff --git a/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt/sling/resources.py b/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt/sling/resources.py
index e06549a48e86f..c0d57e6e643ea 100644
--- a/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt/sling/resources.py
+++ b/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt/sling/resources.py
@@ -405,15 +405,15 @@ def _replicate(
object_key = stream.get("config", {}).get("object")
destination_stream_name = object_key or stream["name"]
- relation_identifier = None
+ table_name = None
if destination_name and destination_stream_name:
- relation_identifier = ".".join([destination_name, destination_stream_name])
+ table_name = ".".join([destination_name, destination_stream_name])
metadata = {
"elapsed_time": end_time - start_time,
"stream_name": stream["name"],
**TableMetadataSet(
- relation_identifier=relation_identifier,
+ table_name=table_name,
),
}
diff --git a/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt_tests/dlt_tests/test_asset_decorator.py b/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt_tests/dlt_tests/test_asset_decorator.py
index 0ed2882b21efa..afcb0c2f8e2a0 100644
--- a/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt_tests/dlt_tests/test_asset_decorator.py
+++ b/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt_tests/dlt_tests/test_asset_decorator.py
@@ -110,7 +110,7 @@ def example_pipeline_assets(
assert repos_materialization.metadata["dagster/row_count"] == IntMetadataValue(3)
- assert repos_materialization.metadata["dagster/relation_identifier"] == TextMetadataValue(
+ assert repos_materialization.metadata["dagster/table_name"] == TextMetadataValue(
text="duckdb.pipeline.repos"
)
diff --git a/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt_tests/sling_tests/test_asset_decorator.py b/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt_tests/sling_tests/test_asset_decorator.py
index 909559cf070b5..87f06916619c8 100644
--- a/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt_tests/sling_tests/test_asset_decorator.py
+++ b/python_modules/libraries/dagster-embedded-elt/dagster_embedded_elt_tests/sling_tests/test_asset_decorator.py
@@ -473,7 +473,7 @@ def my_sling_assets(
}
-def test_relation_identifier(
+def test_table_name(
csv_to_sqlite_dataworks_replication: SlingReplicationParam,
path_to_temp_sqlite_db: str,
):
@@ -501,5 +501,5 @@ def my_sling_assets(context: AssetExecutionContext, sling: SlingResource):
asset_materializations = res.get_asset_materialization_events()
assert len(asset_materializations) == 1
assert asset_materializations[0].materialization.metadata[
- "dagster/relation_identifier"
+ "dagster/table_name"
] == TextMetadataValue(text="SLING_SQLITE.main.orders")
diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran/utils.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran/utils.py
index 194bbfb002f55..ffbdd86f5b3fe 100644
--- a/python_modules/libraries/dagster-fivetran/dagster_fivetran/utils.py
+++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran/utils.py
@@ -29,7 +29,7 @@ def metadata_for_table(
) -> RawMetadataMapping:
metadata: Dict[str, MetadataValue] = {"connector_url": MetadataValue.url(connector_url)}
column_schema = None
- relation_identifier = None
+ table_name = None
if table_data.get("columns"):
columns = check.dict_elem(table_data, "columns")
table_columns = sorted(
@@ -46,9 +46,9 @@ def metadata_for_table(
metadata["column_info"] = MetadataValue.json(columns)
if database and schema and table:
- relation_identifier = ".".join([database, schema, table])
+ table_name = ".".join([database, schema, table])
metadata = {
- **TableMetadataSet(column_schema=column_schema, relation_identifier=relation_identifier),
+ **TableMetadataSet(column_schema=column_schema, table_name=table_name),
**metadata,
}
diff --git a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_load_from_instance.py b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_load_from_instance.py
index 872e38412337d..ac9b16507eea3 100644
--- a/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_load_from_instance.py
+++ b/python_modules/libraries/dagster-fivetran/dagster_fivetran_tests/test_load_from_instance.py
@@ -213,7 +213,7 @@ def downstream_asset(xyz):
), str(assets_def.metadata_by_key)
for key, metadata in assets_def.metadata_by_key.items():
- assert metadata.get("dagster/relation_identifier") == (
+ assert metadata.get("dagster/table_name") == (
"example_database." + ".".join(key.path[-2:])
)
assert has_kind(assets_def.tags_by_key[key], "snowflake")
diff --git a/python_modules/libraries/dagster-gcp-pandas/dagster_gcp_pandas_tests/bigquery/test_type_handler.py b/python_modules/libraries/dagster-gcp-pandas/dagster_gcp_pandas_tests/bigquery/test_type_handler.py
index 2ef95ac030d71..67f007b3f11ed 100644
--- a/python_modules/libraries/dagster-gcp-pandas/dagster_gcp_pandas_tests/bigquery/test_type_handler.py
+++ b/python_modules/libraries/dagster-gcp-pandas/dagster_gcp_pandas_tests/bigquery/test_type_handler.py
@@ -79,7 +79,7 @@ def my_pandas_df() -> pd.DataFrame:
assert len(mats) == 1
mat = mats[0]
- assert mat.materialization.metadata["dagster/relation_identifier"] == MetadataValue.text(
+ assert mat.materialization.metadata["dagster/table_name"] == MetadataValue.text(
f"{os.getenv('GCP_PROJECT_ID')}.{SCHEMA}.{table_name}"
)
diff --git a/python_modules/libraries/dagster-sdf/dagster_sdf/sdf_information_schema.py b/python_modules/libraries/dagster-sdf/dagster_sdf/sdf_information_schema.py
index 4bda70fcf6355..173416b39c0c3 100644
--- a/python_modules/libraries/dagster-sdf/dagster_sdf/sdf_information_schema.py
+++ b/python_modules/libraries/dagster-sdf/dagster_sdf/sdf_information_schema.py
@@ -180,7 +180,7 @@ def build_sdf_multi_asset_args(
column_schema=TableSchema(
columns=table_columns.get(table_row["table_id"], []),
),
- relation_identifier=table_row["table_id"],
+ table_name=table_row["table_id"],
),
**(code_references if code_references else {}),
DAGSTER_SDF_TABLE_ID: table_row["table_id"],
@@ -323,7 +323,7 @@ def stream_asset_observations(
column_schema=TableSchema(
columns=table_columns.get(table_row["table_id"], []),
),
- relation_identifier=table_row["table_id"],
+ table_name=table_row["table_id"],
),
**(code_references if code_references else {}),
DAGSTER_SDF_TABLE_ID: table_row["table_id"],
diff --git a/python_modules/libraries/dagster-snowflake-pandas/dagster_snowflake_pandas_tests/test_snowflake_pandas_type_handler.py b/python_modules/libraries/dagster-snowflake-pandas/dagster_snowflake_pandas_tests/test_snowflake_pandas_type_handler.py
index a869f15a093b3..27e5a390ece1b 100644
--- a/python_modules/libraries/dagster-snowflake-pandas/dagster_snowflake_pandas_tests/test_snowflake_pandas_type_handler.py
+++ b/python_modules/libraries/dagster-snowflake-pandas/dagster_snowflake_pandas_tests/test_snowflake_pandas_type_handler.py
@@ -232,7 +232,7 @@ def my_pandas_df():
assert len(mats) == 1
mat = mats[0]
- assert mat.materialization.metadata["dagster/relation_identifier"] == MetadataValue.text(
+ assert mat.materialization.metadata["dagster/table_name"] == MetadataValue.text(
f"{DATABASE}.{SCHEMA}.{table_name}"
)
diff --git a/python_modules/libraries/dagster-snowflake-pyspark/dagster_snowflake_pyspark_tests/test_snowflake_pyspark_type_handler.py b/python_modules/libraries/dagster-snowflake-pyspark/dagster_snowflake_pyspark_tests/test_snowflake_pyspark_type_handler.py
index 5c8586a10d62d..3465d68da3b5a 100644
--- a/python_modules/libraries/dagster-snowflake-pyspark/dagster_snowflake_pyspark_tests/test_snowflake_pyspark_type_handler.py
+++ b/python_modules/libraries/dagster-snowflake-pyspark/dagster_snowflake_pyspark_tests/test_snowflake_pyspark_type_handler.py
@@ -211,7 +211,7 @@ def my_spark_df():
assert len(mats) == 1
mat = mats[0]
- assert mat.materialization.metadata["dagster/relation_identifier"] == MetadataValue.text(
+ assert mat.materialization.metadata["dagster/table_name"] == MetadataValue.text(
f"{DATABASE}.{SCHEMA}.{table_name}"
)
|