forked from dagster-io/dagster
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[10/n][dagster-fivetran] Implement fivetran_assets and build_fivetran…
…_assets_definitions (dagster-io#25944) ## Summary & Motivation This PR implements the `fivetran_assets` decorator and the `build_fivetran_assets_definitions` factory. - `fivetran_assets` can be used to create all assets for a given Fivetran connector, i.e. one asset per table in the connector. - `build_fivetran_assets_definitions` can be used to create all Fivetran assets defs, one per connector. It uses `fivetran_assets`. Both the asset decorator and factory use `load_fivetran_asset_specs`. This is motivated by the current implementation of `dagster-dbt`, `dagster-dlt` and `dagster-sling` - each leverages an asset decorator that loads the asset specs by itself. To avoid calling the Fivetran API each time `load_fivetran_asset_specs` is called, it is cached using `functools.lru_cache`. `load_fivetran_asset_specs` uses the state-backed defs, so reloading the code won't make additional calls to the Fivetran API, but calling `load_fivetran_asset_specs` N times in a script will make N calls to the Fivetran API. The goals here are: - make the Fivetran integration as similar as possible to the other ELT integrations by using the same patterns, eg. asset decorator - make the user experience as simple as possible and avoid having users manage the asset specs and number of calls to the Fivetran API. ## How I Tested These Changes Additional unit tests with BK. ## Changelog [dagster-fivetran] The `fivetran_assets` decorator is added. It can be used with the `FivetranWorkspace` resource and `DagsterFivetranTranslator` translator to load Fivetran tables for a given connector as assets in Dagster. The `build_fivetran_assets_definitions` factory can be used to create assets for all the connectors in your Fivetran workspace.
- Loading branch information
1 parent
912f0a0
commit 83220c8
Showing
7 changed files
with
363 additions
and
4 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
115 changes: 115 additions & 0 deletions
115
python_modules/libraries/dagster-fivetran/dagster_fivetran/asset_decorator.py
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,115 @@ | ||
from typing import Any, Callable, Optional | ||
|
||
from dagster import AssetsDefinition, multi_asset | ||
from dagster._annotations import experimental | ||
|
||
from dagster_fivetran.resources import FivetranWorkspace | ||
from dagster_fivetran.translator import DagsterFivetranTranslator, FivetranMetadataSet | ||
|
||
|
||
@experimental | ||
def fivetran_assets( | ||
*, | ||
connector_id: str, | ||
workspace: FivetranWorkspace, | ||
name: Optional[str] = None, | ||
group_name: Optional[str] = None, | ||
dagster_fivetran_translator: Optional[DagsterFivetranTranslator] = None, | ||
) -> Callable[[Callable[..., Any]], AssetsDefinition]: | ||
"""Create a definition for how to sync the tables of a given Fivetran connector. | ||
Args: | ||
connector_id (str): The Fivetran Connector ID. You can retrieve this value from the | ||
"Setup" tab of a given connector in the Fivetran UI. | ||
workspace (FivetranWorkspace): The Fivetran workspace to fetch assets from. | ||
name (Optional[str], optional): The name of the op. | ||
group_name (Optional[str], optional): The name of the asset group. | ||
dagster_fivetran_translator (Optional[DagsterFivetranTranslator], optional): The translator to use | ||
to convert Fivetran content into :py:class:`dagster.AssetSpec`. | ||
Defaults to :py:class:`DagsterFivetranTranslator`. | ||
Examples: | ||
Sync the tables of a Fivetran connector: | ||
.. code-block:: python | ||
from dagster_fivetran import FivetranWorkspace, fivetran_assets | ||
import dagster as dg | ||
fivetran_workspace = FivetranWorkspace( | ||
account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"), | ||
api_key=dg.EnvVar("FIVETRAN_API_KEY"), | ||
api_secret=dg.EnvVar("FIVETRAN_API_SECRET"), | ||
) | ||
@fivetran_assets( | ||
connector_id="fivetran_connector_id", | ||
name="fivetran_connector_id", | ||
group_name="fivetran_connector_id", | ||
workspace=fivetran_workspace, | ||
) | ||
def fivetran_connector_assets(context: dg.AssetExecutionContext, fivetran: FivetranWorkspace): | ||
yield from fivetran.sync_and_poll(context=context) | ||
defs = dg.Definitions( | ||
assets=[fivetran_connector_assets], | ||
resources={"fivetran": fivetran_workspace}, | ||
) | ||
Sync the tables of a Fivetran connector with a custom translator: | ||
.. code-block:: python | ||
from dagster_fivetran import ( | ||
DagsterFivetranTranslator, | ||
FivetranConnectorTableProps, | ||
FivetranWorkspace, | ||
fivetran_assets | ||
) | ||
import dagster as dg | ||
from dagster._core.definitions.asset_spec import replace_attributes | ||
class CustomDagsterFivetranTranslator(DagsterFivetranTranslator): | ||
def get_asset_spec(self, props: FivetranConnectorTableProps) -> dg.AssetSpec: | ||
asset_spec = super().get_asset_spec(props) | ||
return replace_attributes( | ||
asset_spec, | ||
key=asset_spec.key.with_prefix("my_prefix"), | ||
) | ||
fivetran_workspace = FivetranWorkspace( | ||
account_id=dg.EnvVar("FIVETRAN_ACCOUNT_ID"), | ||
api_key=dg.EnvVar("FIVETRAN_API_KEY"), | ||
api_secret=dg.EnvVar("FIVETRAN_API_SECRET"), | ||
) | ||
@fivetran_assets( | ||
connector_id="fivetran_connector_id", | ||
name="fivetran_connector_id", | ||
group_name="fivetran_connector_id", | ||
workspace=fivetran_workspace, | ||
dagster_fivetran_translator=CustomDagsterFivetranTranslator(), | ||
) | ||
def fivetran_connector_assets(context: dg.AssetExecutionContext, fivetran: FivetranWorkspace): | ||
yield from fivetran.sync_and_poll(context=context) | ||
defs = dg.Definitions( | ||
assets=[fivetran_connector_assets], | ||
resources={"fivetran": fivetran_workspace}, | ||
) | ||
""" | ||
return multi_asset( | ||
name=name, | ||
group_name=group_name, | ||
can_subset=True, | ||
specs=[ | ||
spec | ||
for spec in workspace.load_asset_specs( | ||
dagster_fivetran_translator=dagster_fivetran_translator | ||
or DagsterFivetranTranslator() | ||
) | ||
if FivetranMetadataSet.extract(spec.metadata).connector_id == connector_id | ||
], | ||
) |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.