Skip to main content

dlt (dagster-dlt)

This library provides a Dagster integration with dlt.

For more information on getting started, see the dlt & Dagster documentation.

Assets

@dagster_dlt.dlt_assets

Asset Factory for using data load tool (dlt).

Parameters:

  • dlt_source (DltSource) – The DltSource to be ingested.
  • dlt_pipeline (Pipeline) – The dlt Pipeline defining the destination parameters.
  • name (Optional[str], optional) – The name of the op.
  • group_name (Optional[str], optional) – The name of the asset group.
  • dagster_dlt_translator (DagsterDltTranslator, optional) – Customization object for defining asset parameters from dlt resources.
  • partitions_def (Optional[PartitionsDefinition]) – Optional partitions definition.
  • op_tags (Optional[Mapping[str, Any]]) – The tags for the underlying op.

Examples:

Loading Hubspot data to Snowflake with an auto materialize policy using the dlt verified source:

from dagster_dlt import DagsterDltResource, DagsterDltTranslator, dlt_assets


class HubspotDagsterDltTranslator(DagsterDltTranslator):
@public
def get_auto_materialize_policy(self, resource: DltResource) -> Optional[AutoMaterializePolicy]:
return AutoMaterializePolicy.eager().with_rules(
AutoMaterializeRule.materialize_on_cron("0 0 * * *")
)


@dlt_assets(
dlt_source=hubspot(include_history=True),
dlt_pipeline=pipeline(
pipeline_name="hubspot",
dataset_name="hubspot",
destination="snowflake",
progress="log",
),
name="hubspot",
group_name="hubspot",
dagster_dlt_translator=HubspotDagsterDltTranslator(),
)
def hubspot_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
yield from dlt.run(context=context)

Loading Github issues to snowflake:

from dagster_dlt import DagsterDltResource, dlt_assets


@dlt_assets(
dlt_source=github_reactions(
"dagster-io", "dagster", items_per_page=100, max_items=250
),
dlt_pipeline=pipeline(
pipeline_name="github_issues",
dataset_name="github",
destination="snowflake",
progress="log",
),
name="github",
group_name="github",
)
def github_reactions_dagster_assets(context: AssetExecutionContext, dlt: DagsterDltResource):
yield from dlt.run(context=context)
dagster_dlt.build_dlt_asset_specs

Build a list of asset specs from a dlt source and pipeline.

Parameters:

  • dlt_source (DltSource) – dlt source object
  • dlt_pipeline (Pipeline) – dlt pipeline object
  • dagster_dlt_translator (Optional[DagsterDltTranslator]) – Allows customizing how to

Returns: List[AssetSpec] list of asset specs from dlt source and pipeline

class dagster_dlt.DagsterDltTranslator
get_asset_key

Defines asset key for a given dlt resource key and dataset name.

This method can be overriden to provide custom asset key for a dlt resource.

Parameters: resource (DltResource) – dlt resourceReturns: AssetKey of Dagster asset derived from dlt resource

get_auto_materialize_policy

Defines resource specific auto materialize policy.

This method can be overriden to provide custom auto materialize policy for a dlt resource.

Parameters: resource (DltResource) – dlt resourceReturns: The automaterialize policy for a resourceReturn type: Optional[AutoMaterializePolicy]

get_automation_condition

Defines resource specific automation condition.

This method can be overridden to provide custom automation condition for a dlt resource.

Parameters: resource (DltResource) – dlt resourceReturns: The automation condition for a resourceReturn type: Optional[AutomationCondition]

get_deps_asset_keys

Defines upstream asset dependencies given a dlt resource.

Defaults to a concatenation of resource.source_name and resource.name.

Parameters: resource (DltResource) – dlt resourceReturns: The Dagster asset keys upstream of dlt_resource_key.Return type: Iterable[AssetKey]

get_description

A method that takes in a dlt resource returns the Dagster description of the resource.

This method can be overriden to provide a custom description for a dlt resource.

Parameters: resource (DltResource) – dlt resourceReturns: The Dagster description for the dlt resource.Return type: Optional[str]

get_group_name

A method that takes in a dlt resource and returns the Dagster group name of the resource.

This method can be overriden to provide a custom group name for a dlt resource.

Parameters: resource (DltResource) – dlt resourceReturns: A Dagster group name for the dlt resource.Return type: Optional[str]

get_kinds

A method that takes in a dlt resource and returns the kinds which should be attached. Defaults to the destination type and “dlt”.

This method can be overriden to provide custom kinds for a dlt resource.

Parameters: resource (DltResource) – dlt resourceReturns: The kinds of the asset.Return type: Set[str]

get_metadata

Defines resource specific metadata.

Parameters: resource (DltResource) – dlt resourceReturns: The custom metadata entries for this resource.Return type: Mapping[str, Any]

get_owners

A method that takes in a dlt resource and returns the Dagster owners of the resource.

This method can be overriden to provide custom owners for a dlt resource.

Parameters: resource (DltResource) – dlt resourceReturns: A sequence of Dagster owners for the dlt resource.Return type: Optional[Sequence[str]]

get_tags

A method that takes in a dlt resource and returns the Dagster tags of the structure.

This method can be overriden to provide custom tags for a dlt resource.

Parameters: resource (DltResource) – dlt resourceReturns: A dictionary representing the Dagster tags for the dlt resource.

Return type: Optional[Mapping[str, str]]

Resources

class dagster_dlt.DagsterDltResource
experimental

This API may break in future versions, even between dot releases.

run

Runs the dlt pipeline with subset support.

Parameters:

  • context (Union[OpExecutionContext, AssetExecutionContext]) – Asset or op execution context
  • dlt_source (Optional[DltSource]) – optional dlt source if resource is used from an @op
  • dlt_pipeline (Optional[Pipeline]) – optional dlt pipeline if resource is used from an @op
  • dagster_dlt_translator (Optional[DagsterDltTranslator]) – optional dlt translator if resource is used from an @op
  • **kwargs (dict[str, Any]) – Keyword args passed to pipeline run method

Returns: An iterator of MaterializeResult or AssetMaterializationReturn type: DltEventIterator[DltEventType]