├── CLAUDE.md ├── Makefile ├── README.md ├── dagster_open_platform ├── definitions.py ├── defs │ ├── __init__.py │ ├── aws │ │ ├── assets.py │ │ ├── constants.py │ │ ├── partitions.py │ │ ├── resources.py │ │ ├── schedules.py │ │ ├── sensors.py │ │ └── utils.py │ ├── claude │ │ ├── README.md │ │ ├── __init__.py │ │ ├── assets.py │ │ ├── definitions.py │ │ ├── partitions.py │ │ ├── resources.py │ │ ├── sql │ │ │ ├── create_cost_table.sql │ │ │ ├── create_usage_table.sql │ │ │ ├── delete_cost_partition.sql │ │ │ ├── delete_usage_partition.sql │ │ │ ├── merge_into_cost.sql │ │ │ └── merge_into_usage.sql │ │ └── utils.py │ ├── common │ │ └── resources.py │ ├── datadog │ │ ├── component │ │ │ ├── component.yaml │ │ │ └── datadog_usage.py │ │ └── py │ │ │ ├── resources.py │ │ │ └── schedule.py │ ├── dbt │ │ ├── __init__.py │ │ ├── assets.py │ │ ├── partitions.py │ │ ├── resources.py │ │ └── schedules.py │ ├── dlt │ │ ├── assets.py │ │ ├── component │ │ │ ├── component.yaml │ │ │ └── loads.py │ │ ├── custom_component.py │ │ ├── resource.py │ │ └── sources │ │ │ ├── __init__.py │ │ │ ├── buildkite │ │ │ └── __init__.py │ │ │ ├── github │ │ │ ├── README.md │ │ │ ├── __init__.py │ │ │ ├── helpers.py │ │ │ ├── queries.py │ │ │ └── settings.py │ │ │ ├── hubspot │ │ │ ├── README.md │ │ │ ├── __init__.py │ │ │ ├── helpers.py │ │ │ └── settings.py │ │ │ ├── local_buildkite.py │ │ │ ├── local_github.py │ │ │ └── thinkific │ │ │ └── __init__.py │ ├── fivetran │ │ ├── components │ │ │ └── component.yaml │ │ └── py │ │ │ └── checks.py │ ├── google_search_console │ │ └── py │ │ │ ├── assets.py │ │ │ ├── definitions.py │ │ │ ├── resources.py │ │ │ └── schedules.py │ ├── hightouch │ │ ├── components │ │ │ └── component.yaml │ │ └── py │ │ │ ├── resources.py │ │ │ └── schedules.py │ ├── hubspot │ │ └── py │ │ │ ├── assets.py │ │ │ └── resources.py │ ├── omni │ │ └── component │ │ │ └── defs.yaml │ ├── pypi │ │ ├── assets │ │ │ ├── daily_file_downloads.py │ │ │ └── weekly_downloads.py │ │ ├── partitions.py │ │ └── resources.py │ ├── schedules │ │ └── component.yaml │ ├── science │ │ ├── definitions.py │ │ └── models │ │ │ └── opportunity_win_probability │ │ │ ├── README.md │ │ │ ├── model_opportunity_win_probability.py │ │ │ ├── production_models │ │ │ └── production_xgb_model_20250725_153354.pkl │ │ │ ├── sql │ │ │ └── all_data.sql │ │ │ └── utils.py │ ├── scout │ │ ├── assets │ │ │ ├── github_issues.py │ │ │ └── scout_run_logs.py │ │ ├── resources │ │ │ ├── github_resource.py │ │ │ └── scoutos_resource.py │ │ └── schedules.py │ ├── segment │ │ └── components │ │ │ └── component.yaml │ ├── sequel │ │ └── py │ │ │ ├── assets.py │ │ │ ├── definitions.py │ │ │ ├── resources.py │ │ │ └── schedules.py │ ├── slack │ │ └── py │ │ │ └── assets │ │ │ ├── dop_health_report.py │ │ │ └── member_metrics.py │ ├── sling │ │ ├── components │ │ │ └── component.yaml │ │ ├── configs │ │ │ ├── cloud_product │ │ │ │ ├── main_event_log.yaml │ │ │ │ ├── main_full_refresh.yaml │ │ │ │ ├── main_high_volume.yaml │ │ │ │ ├── main_low_volume.yaml │ │ │ │ ├── main_runs.yaml │ │ │ │ ├── main_user_event_log.yaml │ │ │ │ ├── shard1_event_log.yaml │ │ │ │ ├── shard1_high_volume.yaml │ │ │ │ ├── shard1_low_volume.yaml │ │ │ │ ├── shard1_runs.yaml │ │ │ │ ├── shard1_user_event_log.yaml │ │ │ │ └── xregion_low_volume.yaml │ │ │ ├── compass │ │ │ │ ├── prod_replication.yaml │ │ │ │ └── staging_replication.yaml │ │ │ └── reporting_db │ │ │ │ ├── enterprise_contract_metadata.yaml │ │ │ │ ├── salesforce_contract_info.yaml │ │ │ │ └── usage_metrics_daily_jobs_aggregated.yaml │ │ └── py │ │ │ ├── jobs.py │ │ │ ├── resources.py │ │ │ ├── schedules.py │ │ │ └── template_vars.py │ ├── snowflake │ │ ├── components │ │ │ ├── common_room │ │ │ │ └── component.yaml │ │ │ └── vpc_flow_logs │ │ │ │ └── component.yaml │ │ └── py │ │ │ ├── assets │ │ │ ├── oss_telemetry.py │ │ │ ├── snowflake_clones.py │ │ │ ├── user_roles.py │ │ │ └── workspace_replication.py │ │ │ ├── jobs.py │ │ │ ├── schedules.py │ │ │ └── sensors.py │ ├── statsig │ │ ├── __init__.py │ │ ├── assets.py │ │ ├── resources.py │ │ └── schedules.py │ └── stripe │ │ └── assets.py ├── lib │ ├── __init__.py │ ├── dbt │ │ ├── __init__.py │ │ └── translator.py │ ├── executable_component.py │ ├── fivetran │ │ └── component.py │ ├── hightouch │ │ └── component.py │ ├── omni │ │ └── component.py │ ├── schedule.py │ ├── segment │ │ └── component.py │ ├── sling │ │ ├── cloud_product_ingest.py │ │ ├── egress.py │ │ └── utils.py │ └── snowflake │ │ ├── component.py │ │ └── sql │ │ └── create_object.sql └── utils │ ├── __init__.py │ ├── environment_helpers.py │ ├── github_gql_queries.py │ └── source_code.py ├── dagster_open_platform_dbt ├── .gitignore ├── .sqlfluffignore ├── CLAUDE.md ├── Makefile ├── checks │ ├── descriptions.sql │ └── field_descriptions.sql ├── dbt_project.yml ├── dependencies.yml ├── macros │ ├── clean_trailing_path_values.sql │ ├── clean_utm_term.sql │ ├── coalesce_referrer_from_search.sql │ ├── email_cleaner.sql │ ├── email_validation.sql │ ├── generate_business_unit_flags.sql │ ├── get_custom_database.sql │ ├── get_custom_schema.sql │ ├── get_domain_from_website.sql │ ├── get_page_attribution_category.sql │ ├── hash_string.sql │ ├── html_cleaner.sql │ ├── is_engaged.sql │ ├── is_existing_business.sql │ ├── is_intent_score_greater_than_zero.sql │ ├── is_internal_user.sql │ ├── is_personal_email.sql │ ├── limit_dates_for_dev.sql │ ├── limit_dates_for_insights.sql │ ├── name_cleaner.sql │ ├── parse_domain_from_email.sql │ ├── query_to_list.sql │ ├── sessionize_page_visits.sql │ └── uri_percent_decode.sql ├── package-lock.yml ├── profiles.yml ├── selectors.yml ├── snapshots │ └── purina │ │ └── snapshots │ │ ├── business │ │ └── arr_by_month_daily_snapshot.sql │ │ ├── hubspot │ │ ├── hubspot_company_daily_snapshot.sql │ │ └── hubspot_contact_daily_snapshot.sql │ │ ├── product │ │ ├── deployments_snapshot.sql │ │ ├── organizations_snapshot.sql │ │ └── user_organizations_snapshot.sql │ │ └── salesforce │ │ ├── salesforce_accounts_daily_snapshot.sql │ │ ├── salesforce_opportunities_daily_snapshot.sql │ │ └── salesforce_users_changes_only_snapshot.sql ├── tests │ ├── generic │ │ ├── test_is_numeric.sql │ │ └── test_is_valid_plan_type.sql │ └── singular │ │ ├── assert_gong_transcript_arrays_have_no_duplicates.sql │ │ ├── assert_other_workflow_type_not_excessive.sql │ │ ├── assert_stripe_id_for_all_customers.sql │ │ └── percent_false_weekly_active_org.sql └── workspace.sdf.yml ├── dagster_open_platform_tests ├── __init__.py ├── __snapshots__ │ └── test_defs.ambr ├── conftest.py ├── test_defs.py ├── test_dlt │ ├── duckdb_with_transformer.py │ └── test_dlt_component.py ├── test_gong │ └── test_basic_execution.py ├── test_snowflake │ ├── __init__.py │ └── test_snowflake_component.py └── utils.py ├── dogfood_master_packages.txt ├── pyproject.toml ├── scripts └── backfill_partitions.py └── uv.lock /CLAUDE.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/CLAUDE.md -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/Makefile -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/README.md -------------------------------------------------------------------------------- /dagster_open_platform/definitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/definitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /dagster_open_platform/defs/aws/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/aws/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/aws/constants.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/aws/constants.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/aws/partitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/aws/partitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/aws/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/aws/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/aws/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/aws/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/aws/sensors.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/aws/sensors.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/aws/utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/aws/utils.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/README.md -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/__init__.py: -------------------------------------------------------------------------------- 1 | # Anthropic/Claude metrics ingestion 2 | 3 | -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/definitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/definitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/partitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/partitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/sql/create_cost_table.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/sql/create_cost_table.sql -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/sql/create_usage_table.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/sql/create_usage_table.sql -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/sql/delete_cost_partition.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/sql/delete_cost_partition.sql -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/sql/delete_usage_partition.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/sql/delete_usage_partition.sql -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/sql/merge_into_cost.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/sql/merge_into_cost.sql -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/sql/merge_into_usage.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/sql/merge_into_usage.sql -------------------------------------------------------------------------------- /dagster_open_platform/defs/claude/utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/claude/utils.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/common/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/common/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/datadog/component/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/datadog/component/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/datadog/component/datadog_usage.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/datadog/component/datadog_usage.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/datadog/py/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/datadog/py/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/datadog/py/schedule.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/datadog/py/schedule.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dbt/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /dagster_open_platform/defs/dbt/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dbt/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dbt/partitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dbt/partitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dbt/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dbt/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dbt/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dbt/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/component/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/component/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/component/loads.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/component/loads.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/custom_component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/custom_component.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/resource.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/resource.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/buildkite/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/buildkite/__init__.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/github/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/github/README.md -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/github/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/github/__init__.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/github/helpers.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/github/helpers.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/github/queries.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/github/queries.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/github/settings.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/github/settings.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/hubspot/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/hubspot/README.md -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/hubspot/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/hubspot/__init__.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/hubspot/helpers.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/hubspot/helpers.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/hubspot/settings.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/hubspot/settings.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/local_buildkite.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/local_buildkite.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/local_github.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/local_github.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/dlt/sources/thinkific/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/dlt/sources/thinkific/__init__.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/fivetran/components/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/fivetran/components/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/fivetran/py/checks.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/fivetran/py/checks.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/google_search_console/py/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/google_search_console/py/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/google_search_console/py/definitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/google_search_console/py/definitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/google_search_console/py/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/google_search_console/py/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/google_search_console/py/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/google_search_console/py/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/hightouch/components/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/hightouch/components/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/hightouch/py/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/hightouch/py/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/hightouch/py/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/hightouch/py/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/hubspot/py/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/hubspot/py/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/hubspot/py/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/hubspot/py/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/omni/component/defs.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/omni/component/defs.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/pypi/assets/daily_file_downloads.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/pypi/assets/daily_file_downloads.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/pypi/assets/weekly_downloads.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/pypi/assets/weekly_downloads.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/pypi/partitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/pypi/partitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/pypi/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/pypi/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/schedules/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/schedules/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/science/definitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/science/definitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/science/models/opportunity_win_probability/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/science/models/opportunity_win_probability/README.md -------------------------------------------------------------------------------- /dagster_open_platform/defs/science/models/opportunity_win_probability/model_opportunity_win_probability.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/science/models/opportunity_win_probability/model_opportunity_win_probability.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/science/models/opportunity_win_probability/production_models/production_xgb_model_20250725_153354.pkl: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/science/models/opportunity_win_probability/production_models/production_xgb_model_20250725_153354.pkl -------------------------------------------------------------------------------- /dagster_open_platform/defs/science/models/opportunity_win_probability/sql/all_data.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/science/models/opportunity_win_probability/sql/all_data.sql -------------------------------------------------------------------------------- /dagster_open_platform/defs/science/models/opportunity_win_probability/utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/science/models/opportunity_win_probability/utils.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/scout/assets/github_issues.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/scout/assets/github_issues.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/scout/assets/scout_run_logs.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/scout/assets/scout_run_logs.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/scout/resources/github_resource.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/scout/resources/github_resource.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/scout/resources/scoutos_resource.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/scout/resources/scoutos_resource.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/scout/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/scout/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/segment/components/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/segment/components/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sequel/py/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sequel/py/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/sequel/py/definitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sequel/py/definitions.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/sequel/py/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sequel/py/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/sequel/py/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sequel/py/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/slack/py/assets/dop_health_report.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/slack/py/assets/dop_health_report.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/slack/py/assets/member_metrics.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/slack/py/assets/member_metrics.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/components/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/components/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/main_event_log.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/main_event_log.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/main_full_refresh.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/main_full_refresh.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/main_high_volume.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/main_high_volume.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/main_low_volume.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/main_low_volume.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/main_runs.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/main_runs.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/main_user_event_log.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/main_user_event_log.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/shard1_event_log.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/shard1_event_log.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/shard1_high_volume.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/shard1_high_volume.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/shard1_low_volume.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/shard1_low_volume.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/shard1_runs.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/shard1_runs.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/shard1_user_event_log.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/shard1_user_event_log.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/cloud_product/xregion_low_volume.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/cloud_product/xregion_low_volume.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/compass/prod_replication.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/compass/prod_replication.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/compass/staging_replication.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/compass/staging_replication.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/reporting_db/enterprise_contract_metadata.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/reporting_db/enterprise_contract_metadata.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/reporting_db/salesforce_contract_info.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/reporting_db/salesforce_contract_info.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/configs/reporting_db/usage_metrics_daily_jobs_aggregated.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/configs/reporting_db/usage_metrics_daily_jobs_aggregated.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/py/jobs.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/py/jobs.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/py/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/py/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/py/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/py/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/sling/py/template_vars.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/sling/py/template_vars.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/components/common_room/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/components/common_room/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/components/vpc_flow_logs/component.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/components/vpc_flow_logs/component.yaml -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/py/assets/oss_telemetry.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/py/assets/oss_telemetry.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/py/assets/snowflake_clones.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/py/assets/snowflake_clones.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/py/assets/user_roles.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/py/assets/user_roles.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/py/assets/workspace_replication.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/py/assets/workspace_replication.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/py/jobs.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/py/jobs.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/py/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/py/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/snowflake/py/sensors.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/snowflake/py/sensors.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/statsig/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /dagster_open_platform/defs/statsig/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/statsig/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/statsig/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/statsig/resources.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/statsig/schedules.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/statsig/schedules.py -------------------------------------------------------------------------------- /dagster_open_platform/defs/stripe/assets.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/defs/stripe/assets.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/__init__.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/dbt/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /dagster_open_platform/lib/dbt/translator.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/dbt/translator.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/executable_component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/executable_component.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/fivetran/component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/fivetran/component.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/hightouch/component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/hightouch/component.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/omni/component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/omni/component.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/schedule.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/schedule.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/segment/component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/segment/component.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/sling/cloud_product_ingest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/sling/cloud_product_ingest.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/sling/egress.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/sling/egress.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/sling/utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/sling/utils.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/snowflake/component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/snowflake/component.py -------------------------------------------------------------------------------- /dagster_open_platform/lib/snowflake/sql/create_object.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/lib/snowflake/sql/create_object.sql -------------------------------------------------------------------------------- /dagster_open_platform/utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /dagster_open_platform/utils/environment_helpers.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/utils/environment_helpers.py -------------------------------------------------------------------------------- /dagster_open_platform/utils/github_gql_queries.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/utils/github_gql_queries.py -------------------------------------------------------------------------------- /dagster_open_platform/utils/source_code.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform/utils/source_code.py -------------------------------------------------------------------------------- /dagster_open_platform_dbt/.gitignore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/.gitignore -------------------------------------------------------------------------------- /dagster_open_platform_dbt/.sqlfluffignore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/.sqlfluffignore -------------------------------------------------------------------------------- /dagster_open_platform_dbt/CLAUDE.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/CLAUDE.md -------------------------------------------------------------------------------- /dagster_open_platform_dbt/Makefile: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/Makefile -------------------------------------------------------------------------------- /dagster_open_platform_dbt/checks/descriptions.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/checks/descriptions.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/checks/field_descriptions.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/checks/field_descriptions.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/dbt_project.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/dbt_project.yml -------------------------------------------------------------------------------- /dagster_open_platform_dbt/dependencies.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/dependencies.yml -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/clean_trailing_path_values.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/clean_trailing_path_values.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/clean_utm_term.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/clean_utm_term.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/coalesce_referrer_from_search.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/coalesce_referrer_from_search.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/email_cleaner.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/email_cleaner.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/email_validation.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/email_validation.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/generate_business_unit_flags.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/generate_business_unit_flags.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/get_custom_database.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/get_custom_database.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/get_custom_schema.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/get_custom_schema.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/get_domain_from_website.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/get_domain_from_website.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/get_page_attribution_category.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/get_page_attribution_category.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/hash_string.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/hash_string.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/html_cleaner.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/html_cleaner.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/is_engaged.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/is_engaged.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/is_existing_business.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/is_existing_business.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/is_intent_score_greater_than_zero.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/is_intent_score_greater_than_zero.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/is_internal_user.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/is_internal_user.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/is_personal_email.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/is_personal_email.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/limit_dates_for_dev.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/limit_dates_for_dev.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/limit_dates_for_insights.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/limit_dates_for_insights.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/name_cleaner.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/name_cleaner.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/parse_domain_from_email.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/parse_domain_from_email.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/query_to_list.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/query_to_list.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/sessionize_page_visits.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/sessionize_page_visits.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/macros/uri_percent_decode.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/macros/uri_percent_decode.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/package-lock.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/package-lock.yml -------------------------------------------------------------------------------- /dagster_open_platform_dbt/profiles.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/profiles.yml -------------------------------------------------------------------------------- /dagster_open_platform_dbt/selectors.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/selectors.yml -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/business/arr_by_month_daily_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/business/arr_by_month_daily_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/hubspot/hubspot_company_daily_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/hubspot/hubspot_company_daily_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/hubspot/hubspot_contact_daily_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/hubspot/hubspot_contact_daily_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/product/deployments_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/product/deployments_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/product/organizations_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/product/organizations_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/product/user_organizations_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/product/user_organizations_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/salesforce/salesforce_accounts_daily_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/salesforce/salesforce_accounts_daily_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/salesforce/salesforce_opportunities_daily_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/salesforce/salesforce_opportunities_daily_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/snapshots/purina/snapshots/salesforce/salesforce_users_changes_only_snapshot.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/snapshots/purina/snapshots/salesforce/salesforce_users_changes_only_snapshot.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/tests/generic/test_is_numeric.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/tests/generic/test_is_numeric.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/tests/generic/test_is_valid_plan_type.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/tests/generic/test_is_valid_plan_type.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/tests/singular/assert_gong_transcript_arrays_have_no_duplicates.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/tests/singular/assert_gong_transcript_arrays_have_no_duplicates.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/tests/singular/assert_other_workflow_type_not_excessive.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/tests/singular/assert_other_workflow_type_not_excessive.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/tests/singular/assert_stripe_id_for_all_customers.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/tests/singular/assert_stripe_id_for_all_customers.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/tests/singular/percent_false_weekly_active_org.sql: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/tests/singular/percent_false_weekly_active_org.sql -------------------------------------------------------------------------------- /dagster_open_platform_dbt/workspace.sdf.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_dbt/workspace.sdf.yml -------------------------------------------------------------------------------- /dagster_open_platform_tests/__init__.py: -------------------------------------------------------------------------------- 1 | 2 | -------------------------------------------------------------------------------- /dagster_open_platform_tests/__snapshots__/test_defs.ambr: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_tests/__snapshots__/test_defs.ambr -------------------------------------------------------------------------------- /dagster_open_platform_tests/conftest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_tests/conftest.py -------------------------------------------------------------------------------- /dagster_open_platform_tests/test_defs.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_tests/test_defs.py -------------------------------------------------------------------------------- /dagster_open_platform_tests/test_dlt/duckdb_with_transformer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_tests/test_dlt/duckdb_with_transformer.py -------------------------------------------------------------------------------- /dagster_open_platform_tests/test_dlt/test_dlt_component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_tests/test_dlt/test_dlt_component.py -------------------------------------------------------------------------------- /dagster_open_platform_tests/test_gong/test_basic_execution.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_tests/test_gong/test_basic_execution.py -------------------------------------------------------------------------------- /dagster_open_platform_tests/test_snowflake/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /dagster_open_platform_tests/test_snowflake/test_snowflake_component.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_tests/test_snowflake/test_snowflake_component.py -------------------------------------------------------------------------------- /dagster_open_platform_tests/utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dagster_open_platform_tests/utils.py -------------------------------------------------------------------------------- /dogfood_master_packages.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/dogfood_master_packages.txt -------------------------------------------------------------------------------- /pyproject.toml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/pyproject.toml -------------------------------------------------------------------------------- /scripts/backfill_partitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/scripts/backfill_partitions.py -------------------------------------------------------------------------------- /uv.lock: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/dagster-io/dagster-open-platform/HEAD/uv.lock --------------------------------------------------------------------------------