This guide provides instructions for using Dagster with Power BI. Your Power BI assets, such as semantic models, data sources, reports, and dashboards, can be represented in the Dagster asset graph, allowing you to track lineage and dependencies between Power BI assets and upstream data assets you are already modeling in Dagster. You can also use Dagster to orchestrate Power BI semantic models, allowing you to trigger refreshes of these models on a cadence or based on upstream data changes.
To load Power BI assets into the Dagster asset graph, you must first construct a PowerBIWorkspace resource, which allows Dagster to communicate with your Power BI workspace. You'll need to supply your workspace ID and credentials. You may configure a service principal or use an API access token, which can be passed directly or accessed from the environment using EnvVar.
Dagster can automatically load all semantic models, data sources, reports, and dashboards from your Power BI workspace. Call the build_defs() function, which returns a Definitions object containing all the asset definitions for these Power BI assets.
import uuid
from http import client
from typing import cast
from dagster_powerbi import PowerBIServicePrincipal, PowerBIToken, PowerBIWorkspace
from dagster import Definitions, EnvVar, asset, define_asset_job
# Connect using a service principal
resource = PowerBIWorkspace(
credentials=PowerBIServicePrincipal(
client_id=EnvVar("POWER_BI_CLIENT_ID"),
client_secret=EnvVar("POWER_BI_CLIENT_SECRET"),
tenant_id=EnvVar("POWER_BI_TENANT_ID"),),
workspace_id=EnvVar("POWER_BI_WORKSPACE_ID"),)# Alternatively, connect directly using an API access token
resource = PowerBIWorkspace(
credentials=PowerBIToken(api_token=EnvVar("POWER_BI_API_TOKEN")),
workspace_id=EnvVar("POWER_BI_WORKSPACE_ID"),)
defs = resource.build_defs()
Customize asset definition metadata for Power BI assets#
By default, Dagster will generate asset keys for each Power BI asset based on its type and name and populate default metadata. You can further customize asset properties by using Definitions.map_asset_specs, which applies a function of your choosing to each asset.
from dagster_powerbi import PowerBIServicePrincipal, PowerBIWorkspace
from dagster_powerbi.translator import PowerBIContentData
from dagster import EnvVar
from dagster._core.definitions.asset_key import AssetKey
from dagster._core.definitions.asset_spec import AssetSpec
resource = PowerBIWorkspace(
credentials=PowerBIServicePrincipal(
client_id=EnvVar("POWER_BI_CLIENT_ID"),
client_secret=EnvVar("POWER_BI_CLIENT_SECRET"),
tenant_id=EnvVar("POWER_BI_TENANT_ID"),),
workspace_id=EnvVar("POWER_BI_WORKSPACE_ID"),)defcustomize_powerbi_asset(spec: AssetSpec)-> AssetSpec:# We prefix all dashboard asset keys with "powerbi-dashboard" for# organizational purposes
key = spec.key
if spec.metadata["dagster-powerbi/asset-type"]=="dashboard":
key = key.with_prefix("powerbi-dashboard")return spec._replace(key=key, owners="my_team")
defs = resource.build_defs().map_asset_specs(customize_powerbi_asset)
Definitions from multiple Power BI workspaces can be combined by instantiating multiple PowerBIWorkspace resources and merging their definitions. This lets you view all your Power BI assets in a single asset graph:
from dagster_powerbi import PowerBIServicePrincipal, PowerBIWorkspace
from dagster import Definitions, EnvVar
credentials = PowerBIServicePrincipal(
client_id=EnvVar("POWER_BI_CLIENT_ID"),
client_secret=EnvVar("POWER_BI_CLIENT_SECRET"),
tenant_id=EnvVar("POWER_BI_TENANT_ID"),)
sales_team_workspace = PowerBIWorkspace(
credentials=credentials,
workspace_id="726c94ff-c408-4f43-8edf-61fbfa1753c7",)
marketing_team_workspace = PowerBIWorkspace(
credentials=credentials,
workspace_id="8b7f815d-4e64-40dd-993c-cfa4fb12edee",)# We use Definitions.merge to combine the definitions from both workspaces# into a single set of definitions to load
defs = Definitions.merge(
sales_team_workspace.build_defs(),
marketing_team_workspace.build_defs(),)
Materialize Power BI semantic models from Dagster#
Dagster's default behavior is to pull in representations of Power BI semantic models as external assets, which appear in the asset graph but can't be materialized. However, you can instruct Dagster to allow you to materialize these semantic models, refreshing them, by passing enable_refresh_semantic_models=True to the build_defs() function:
import uuid
from typing import cast
from dagster_powerbi import PowerBIServicePrincipal, PowerBIWorkspace
from dagster import Definitions, EnvVar, asset, define_asset_job
resource = PowerBIWorkspace(
credentials=PowerBIServicePrincipal(
client_id=EnvVar("POWER_BI_CLIENT_ID"),
client_secret=EnvVar("POWER_BI_CLIENT_SECRET"),
tenant_id=EnvVar("POWER_BI_TENANT_ID"),),
workspace_id=EnvVar("POWER_BI_WORKSPACE_ID"),)
defs = resource.build_defs(enable_refresh_semantic_models=True)
You can then add these semantic models to jobs or as targets of Dagster sensors or schedules to trigger refreshes of the models on a cadence or based on other conditions.