Skip to content

Scheduler crashes with InvalidStatsNameException for non-ASCII DAG/TaskGroup names when OTel metrics are enabled #68018

@ersalil

Description

@ersalil

Under which category would you file this issue?

Airflow Core

Apache Airflow version

3.x (confirmed on Airflow v3.1.8)

What happened and how to reproduce it?

When OTel metrics are enabled and a DAG contains a TaskGroup or Task ID
with a non-ASCII character (e.g. ç, ã, é), the scheduler enters
CrashLoopBackOff on every restart instead of skipping the invalid metric
and continuing normally.

The crash happens in the scheduler's critical section when it tries to
queue task instances and emit a state-change metric for them:

File "airflow/jobs/scheduler_job_runner.py", in _executable_task_instances_to_queued
ti.emit_state_change_metric(TaskInstanceState.QUEUED)
File "airflow/models/taskinstance.py", in emit_state_change_metric
Stats.timing(f"dag.{self.dag_id}.{self.task_id}.{metric_name}", timing)
File "airflow_shared/observability/metrics/otel_logger.py", in timing
if self.metrics_validator.test(stat) and name_is_otel_safe(self.prefix, stat):
File "airflow_shared/observability/metrics/otel_logger.py", in name_is_otel_safe
return bool(stat_name_otel_handler(prefix, name, max_length=OTEL_NAME_MAX_LENGTH))
File "airflow_shared/observability/metrics/validators.py", in stat_name_otel_handler
stat_name_default_handler(proposed_stat_name, ...)
File "airflow_shared/observability/metrics/validators.py", in stat_name_default_handler
raise InvalidStatsNameException(...)

airflow.exceptions.InvalidStatsNameException: The stat name
(airflow.dag.sql_server_el.kmt_lista_preços_tasks.run_kmt_lista_precos_pipeline.scheduled_duration)
has to be composed of ASCII alphabets, numbers, or the underscore, dot, or dash characters.

How to reproduce

  1. Enable OTel metrics on an Airflow 3.x deployment.
  2. Create a DAG with a TaskGroup or task ID containing a non-ASCII character
    (e.g. ç, ã, ö):
with TaskGroup("kmt_lista_preços_tasks") as tg:
    ...
  1. Let the DAG create a scheduled run so task instances exist in the database.
  2. Observe the scheduler entering CrashLoopBackOff with InvalidStatsNameException.

What you think should happen instead?

name_is_otel_safe() has a return type of -> bool. A function with that
contract must never raise, it should return False for invalid names and
allow the caller to skip the metric. The scheduler must never crash due to a
metric emission failure.

The validation logic in stat_name_otel_handler and stat_name_default_handler
is correct, non-ASCII characters are genuinely invalid per the OTel instrument
name spec. The problem is that name_is_otel_safe() does not catch the
InvalidStatsNameException that stat_name_otel_handler is documented to raise.

Operating System

No response

Deployment

Astronomer

Apache Airflow Provider(s)

No response

Versions of Apache Airflow Providers

No response

Official Helm Chart version

Not Applicable

Kubernetes Version

No response

Helm Chart configuration

No response

Docker Image customizations

No response

Anything else?

No response

Are you willing to submit PR?

  • Yes I am willing to submit a PR!

Code of Conduct

Metadata

Metadata

Assignees

No one assigned

    Labels

    area:Schedulerincluding HA (high availability) schedulerarea:metricskind:bugThis is a clearly a bugneeds-triagelabel for new issues that we didn't triage yet

    Type

    No type
    No fields configured for issues without a type.

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions