dagster-io / dagster

An orchestration platform for the development, production, and observation of data assets.
https://dagster.io
Apache License 2.0
11.17k stars 1.4k forks source link

[dagster-msteams] make_teams_on_run_failure_sensor monitors all code locations #14353

Closed razafinl closed 6 months ago

razafinl commented 1 year ago

Dagster version

dagster, version 1.3.4

What's the issue?

make_teams_on_run_failure_sensor creates sensor that monitors all code locations regardless of monitor_all_repositories value.

This prevents the ability to isolate this teams_sensor instances per code location for a given Dagster instance.

What did you expect to happen?

When monitor_all_repositories is set to False (default value), only failure from the sensor code location should be caught by teams_on_run_failure sensor.

How to reproduce?

Create an Dagster instance with multiple code locations. Each code location contains a failing job/asset definition with teams_on_run_failure sensor.

from dagster import asset, Definitions, load_assets_from_current_module
from dagster_msteams import make_teams_on_run_failure_sensor
from os import getenv

LOCATION = "a"

@asset(name=f"error_asset_{LOCATION}")
def error_asset() -> None:
    raise Exception(f"Error from location {LOCATION}")

defs = Definitions(
    assets  = load_assets_from_current_module(key_prefix=LOCATION),
    sensors=[
        make_teams_on_run_failure_sensor(
            hook_url=getenv("MS_TEAMS_WEBHOOK"),
            dagit_base_url=getenv("DAGIT_URL"),
        )
    ],
)

When triggering a failing run on location a, all teams_on_run_failure sensors (from locations a and b) are triggered. We are expecting only the sensor from location a to be triggered.

Deployment type

Local

Deployment details

Behavior has been observed from both local (dagster dev) and kubernetes deployments.

Additional information

This behavior is also impacting slack sensors.

Message from the maintainers

Impacted by this issue? Give it a 👍! We factor engagement into prioritization.

jamiedemaria commented 1 year ago

cc @schrockn I think this is a bug caused by all Definitions all having the name __repository__ . In the run_status_sensor we do a check

dagster_run.external_job_origin.external_repository_origin.repository_name == context.repository_name

(source) which would return True for all job executions in any code location since they all have the same name

originally this check was meant to check that the job was executed in the same repository that the sensor is in, but i'm not sure how to update this condition now that all Definitions have the same name

schrockn commented 1 year ago

Hmmm I'm not sure this is caused by Definitions. Definitions just creates a single repository with a single special name. You could construct a repository with that name with the old api.

It was more like this was a pre-existing bug that was more likely to be exposed by the change. If I am reading the code correctly, I think this will happen in any case where there are two repositories in different code locations and the repositories happen to have the same name. I think the solution is make sure that it is in the same code location in addition to having the same repository name, which is code-location-scoped.

GregoireJan commented 1 year ago

This bug affect our operation as well in the same way as describe above. I would add that the sensor works as expected (read "not across all code locations") for the old code location where @repository is still in use instead of Definitions.

emirkmo commented 8 months ago

This can be solved using the CodeLocationFilter but how to find the code location of the sensor without having the user hard code in the possiblities from their configs (say the name is different for local dev vs. in production vs. docker ci dev build, etc.)?