Skip to main content

Cross repository dependencies

One of the most powerful features of multi-repository code locations is the ability to create asset dependencies that span across different repositories and teams. This enables organizations to maintain repository boundaries while still allowing data to flow seamlessly between teams.

Dependency Architecture

The example demonstrates two key cross-repository dependencies:

  • ML → Analytics: customer_features depends on customer_order_summary
  • ML → Analytics: product_features depends on product_performance

These dependencies enable the ML team to build sophisticated features while relying on the Analytics team's data transformations, creating a natural data pipeline across organizational boundaries.

Explicit Dependency Declaration

Cross-repository dependencies are declared explicitly using Dagster's AssetKey mechanism:

features.py - Cross-Repository Dependency Declaration
@asset(
deps=[AssetKey("product_performance")], # Just declares dependency
group_name="ml_features",
)
def product_features() -> pd.DataFrame:
"""Features for product recommendation and demand forecasting models."""
# Load external asset manually - simple and explicit
product_performance = load_external_asset("product_performance")

features = product_performance.copy()

# Calculate velocity metrics
features["revenue_per_order"] = features["revenue"] / features["orders"]
features["units_per_order"] = features["units_sold"] / features["orders"]

# Create category encoding
category_dummies = pd.get_dummies(features["category"], prefix="category")
features = pd.concat([features, category_dummies], axis=1)

# Calculate performance percentiles
features["revenue_percentile"] = features["revenue"].rank(pct=True)
features["profit_percentile"] = features["profit"].rank(pct=True)
features["units_percentile"] = features["units_sold"].rank(pct=True)

# Create performance tiers
def performance_tier(row):
if row["revenue_percentile"] >= 0.8 and row["profit_percentile"] >= 0.8:
return "high_performer"
elif row["revenue_percentile"] >= 0.5 and row["profit_percentile"] >= 0.5:
return "medium_performer"
else:
return "low_performer"

features["performance_tier"] = features.apply(performance_tier, axis=1)

return features

The deps=[AssetKey("customer_order_summary")] declaration serves multiple purposes:

  • Lineage Tracking: Dagster understands the dependency relationship and shows it in the asset graph
  • Execution Ordering: Ensures upstream assets are materialized before downstream assets run
  • Impact Analysis: Changes to upstream assets trigger appropriate downstream re-computation

The dependency is declared by asset key only - the ML repository doesn't need to know implementation details about how the Analytics team produces the data.

Shared Storage Strategy

Cross-repository dependencies require a shared storage layer that both code locations can access:

definitions.py - Shared Storage Configuration
        # Add shared I/O manager for cross-repository access
resources={
**defs_from_folder.resources,
"io_manager": dg.FilesystemIOManager(base_dir="~/Documents/dagster_shared_assets"),
},

Both repositories configure the same FilesystemIOManager with identical base directories. In production environments, this would typically be:

  • AWS S3: S3IOManager with shared bucket access
  • Google Cloud Storage: GCSIOManager with shared bucket access
  • Azure Blob Storage: AzureBlobStorageIOManager with shared container access

Manual Asset Loading Pattern

The ML repository implements a manual loading pattern for external dependencies:

features.py - External Asset Loading Utility
def load_external_asset(asset_name: str) -> pd.DataFrame:
"""Simple utility to load external assets from shared storage."""
file_path = SHARED_DATA_PATH / f"{asset_name}.parquet"
if not file_path.exists():
raise FileNotFoundError(f"External asset {asset_name} not found at {file_path}")
return pd.read_parquet(file_path)

This utility function provides explicit control over how external assets are loaded:

  • Error Handling: Clear error messages when external assets aren't available
  • Path Management: Consistent file path construction for external assets
  • Type Safety: Returns properly typed DataFrames for downstream processing

The manual loading approach gives teams full control over error handling and data validation when consuming external dependencies.

Asset Key Coordination

Successful cross-repository dependencies require careful coordination of asset keys between teams:

analytics_models.py - Producer Asset Definition
@dg.asset(deps=["customer_data", "order_data"], group_name="analytics")
def customer_order_summary(customer_data: pd.DataFrame, order_data: pd.DataFrame) -> pd.DataFrame:
"""Summary of customer orders for analytics."""
# Join customer and order data
summary = (
order_data.groupby("customer_id")
.agg({"order_id": "count", "total_amount": ["sum", "mean"], "order_date": ["min", "max"]})
.round(2)
)

# Flatten column names
summary.columns = [
"total_orders",
"total_spent",
"avg_order_value",
"first_order",
"last_order",
]
summary = summary.reset_index()

# Add customer information
summary = summary.merge(
customer_data[["customer_id", "name", "tier"]], on="customer_id", how="left"
)

return summary

The Analytics team produces customer_order_summary with a specific asset key that the ML team references. This coordination requires:

  • Naming Conventions: Teams must agree on asset naming patterns
  • Key Stability: Asset keys should remain stable across deployments
  • Documentation: Clear documentation of which assets are intended for external consumption

Cross-Repository Execution Flow

When assets have cross-repository dependencies, execution follows a specific pattern:

  1. Analytics Materialization: The Analytics team materializes customer_order_summary
  2. Storage Persistence: The asset is persisted to shared storage via the I/O manager
  3. ML Discovery: The ML repository's asset dependency system recognizes the available upstream asset
  4. ML Execution: The ML team can now materialize customer_features, loading the upstream data

This flow enables asynchronous execution - teams can work independently while ensuring data consistency across repositories.

Local Development and Testing

For local development, the workspace configuration enables testing cross-repository dependencies:

workspace.yaml - Multi-Location Development
load_from:
- python_package:
package_name: analytics.definitions
location_name: analytics-team
working_directory: ./repo-analytics
- python_package:
package_name: ml_platform.definitions
location_name: ml-platform
working_directory: ./repo-ml

Developers can:

  1. Start the workspace: DAGSTER_WORKSPACE_YAML=workspace.yaml dagster-webserver
  2. Materialize analytics assets: Use the UI to materialize upstream assets first
  3. Test ML dependencies: Verify that ML assets can access analytics outputs
  4. View full lineage: See the complete asset graph spanning both repositories

Production Deployment Considerations

In production Dagster+ deployments, cross-repository dependencies require additional considerations:

  • Storage Permissions: Both code locations need read/write access to shared storage
  • Network Connectivity: Code locations must be able to access the same storage systems
  • Deployment Coordination: Teams should coordinate deployment timing to avoid breaking dependencies
  • Monitoring: Set up alerts for failed cross-repository asset materializations

The example's simple file-based approach translates directly to cloud storage solutions with minimal configuration changes, making it easy to evolve from local development to production deployment.

Next steps

Now that we understand how cross-repository dependencies work technically, we can explore how to deploy and manage multiple code locations in production Dagster+ environments.